diff --git hcatalog/build.xml hcatalog/build.xml index 858f001..13c61df 100644 --- hcatalog/build.xml +++ hcatalog/build.xml @@ -223,8 +223,10 @@ + + options.addOption(OptionBuilder + .hasArg() + .withArgName("file") + .withDescription("hcat commands in file") + .create('f')); + + // -g + options.addOption(OptionBuilder + .hasArg(). + withArgName("group"). + withDescription("group for the db/table specified in CREATE statement"). + create('g')); + + // -p + options.addOption(OptionBuilder + .hasArg() + .withArgName("perms") + .withDescription("permissions for the db/table specified in CREATE statement") + .create('p')); + + // -D + options.addOption(OptionBuilder + .hasArgs(2) + .withArgName("property=value") + .withValueSeparator() + .withDescription("use hadoop value for given property") + .create('D')); + + // [-h|--help] + options.addOption(new Option("h", "help", false, "Print help information")); + + Parser parser = new GnuParser(); + CommandLine cmdLine = null; + + try { + cmdLine = parser.parse(options, args); + + } catch (ParseException e) { + printUsage(options, ss.err); + System.exit(1); + } + // -e + String execString = (String) cmdLine.getOptionValue('e'); + // -f + String fileName = (String) cmdLine.getOptionValue('f'); + // -h + if (cmdLine.hasOption('h')) { + printUsage(options, ss.out); + System.exit(0); + } + + if (execString != null && fileName != null) { + ss.err.println("The '-e' and '-f' options cannot be specified simultaneously"); + printUsage(options, ss.err); + System.exit(1); + } + + // -p + String perms = (String) cmdLine.getOptionValue('p'); + if (perms != null) { + validatePermissions(ss, conf, perms); + } + + // -g + String grp = (String) cmdLine.getOptionValue('g'); + if (grp != null) { + conf.set(HCatConstants.HCAT_GROUP, grp); + } + + // -D + setConfProperties(conf, cmdLine.getOptionProperties("D")); + + if (execString != null) { + System.exit(processLine(execString)); + } + + try { + if (fileName != null) { + System.exit(processFile(fileName)); + } + } catch (FileNotFoundException e) { + ss.err.println("Input file not found. (" + e.getMessage() + ")"); + System.exit(1); + } catch (IOException e) { + ss.err.println("Could not open input file for reading. (" + e.getMessage() + ")"); + System.exit(1); + } + + // -h + printUsage(options, ss.err); + System.exit(1); + } + + private static void setConfProperties(HiveConf conf, Properties props) { + for (java.util.Map.Entry e : props.entrySet()) + conf.set((String) e.getKey(), (String) e.getValue()); + } + + private static int processLine(String line) { + int ret = 0; + + String command = ""; + for (String oneCmd : line.split(";")) { + + if (StringUtils.endsWith(oneCmd, "\\")) { + command += StringUtils.chop(oneCmd) + ";"; + continue; + } else { + command += oneCmd; + } + if (StringUtils.isBlank(command)) { + continue; + } + + ret = processCmd(command); + command = ""; + } + return ret; + } + + private static int processFile(String fileName) throws IOException { + FileReader fileReader = null; + BufferedReader reader = null; + try { + fileReader = new FileReader(fileName); + reader = new BufferedReader(fileReader); + String line; + StringBuilder qsb = new StringBuilder(); + + while ((line = reader.readLine()) != null) { + qsb.append(line + "\n"); + } + + return (processLine(qsb.toString())); + } finally { + if (fileReader != null) { + fileReader.close(); + } + if (reader != null) { + reader.close(); + } + } + } + + private static int processCmd(String cmd) { + + SessionState ss = SessionState.get(); + long start = System.currentTimeMillis(); + + cmd = cmd.trim(); + String firstToken = cmd.split("\\s+")[0].trim(); + + if (firstToken.equalsIgnoreCase("set")) { + return new SetProcessor().run(cmd.substring(firstToken.length()).trim()).getResponseCode(); + } else if (firstToken.equalsIgnoreCase("dfs")) { + return new DfsProcessor(ss.getConf()).run(cmd.substring(firstToken.length()).trim()).getResponseCode(); + } + + HCatDriver driver = new HCatDriver(); + + int ret = driver.run(cmd).getResponseCode(); + + if (ret != 0) { + driver.close(); + System.exit(ret); + } + + ArrayList res = new ArrayList(); + try { + while (driver.getResults(res)) { + for (String r : res) { + ss.out.println(r); + } + res.clear(); + } + } catch (IOException e) { + ss.err.println("Failed with exception " + e.getClass().getName() + ":" + + e.getMessage() + "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e)); + ret = 1; + } catch (CommandNeedRetryException e) { + ss.err.println("Failed with exception " + e.getClass().getName() + ":" + + e.getMessage() + "\n" + org.apache.hadoop.util.StringUtils.stringifyException(e)); + ret = 1; + } + + int cret = driver.close(); + if (ret == 0) { + ret = cret; + } + + long end = System.currentTimeMillis(); + if (end > start) { + double timeTaken = (end - start) / 1000.0; + ss.err.println("Time taken: " + timeTaken + " seconds"); + } + return ret; + } + + private static void printUsage(Options options, OutputStream os) { + PrintWriter pw = new PrintWriter(os); + new HelpFormatter().printHelp(pw, 2 * HelpFormatter.DEFAULT_WIDTH, + "hcat { -e \"\" | -f \"\" } [ -g \"\" ] [ -p \"\" ] [ -D\"=\" ]", + null, options, HelpFormatter.DEFAULT_LEFT_PAD, HelpFormatter.DEFAULT_DESC_PAD, + null, false); + pw.flush(); + } + + private static void validatePermissions(CliSessionState ss, HiveConf conf, String perms) { + perms = perms.trim(); + FsPermission fp = null; + + if (perms.matches("^\\s*([r,w,x,-]{9})\\s*$")) { + fp = FsPermission.valueOf("d" + perms); + } else if (perms.matches("^\\s*([0-7]{3})\\s*$")) { + fp = new FsPermission(Short.decode("0" + perms)); + } else { + ss.err.println("Invalid permission specification: " + perms); + System.exit(1); + } + + if (!HCatUtil.validateMorePermissive(fp.getUserAction(), fp.getGroupAction())) { + ss.err.println("Invalid permission specification: " + perms + " : user permissions must be more permissive than group permission "); + System.exit(1); + } + if (!HCatUtil.validateMorePermissive(fp.getGroupAction(), fp.getOtherAction())) { + ss.err.println("Invalid permission specification: " + perms + " : group permissions must be more permissive than other permission "); + System.exit(1); + } + if ((!HCatUtil.validateExecuteBitPresentIfReadOrWrite(fp.getUserAction())) || + (!HCatUtil.validateExecuteBitPresentIfReadOrWrite(fp.getGroupAction())) || + (!HCatUtil.validateExecuteBitPresentIfReadOrWrite(fp.getOtherAction()))) { + ss.err.println("Invalid permission specification: " + perms + " : permissions must have execute permissions if read or write permissions are specified "); + System.exit(1); + } + + conf.set(HCatConstants.HCAT_PERMS, "d" + fp.toString()); + + } + + +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/cli/HCatDriver.java hcatalog/core/src/main/java/org/apache/hcatalog/cli/HCatDriver.java new file mode 100644 index 0000000..d06f555 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/cli/HCatDriver.java @@ -0,0 +1,146 @@ +/** + * 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.hcatalog.cli; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hive.metastore.MetaStoreUtils; +import org.apache.hadoop.hive.metastore.Warehouse; +import org.apache.hadoop.hive.ql.CommandNeedRetryException; +import org.apache.hadoop.hive.ql.Driver; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hcatalog.common.HCatConstants; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.cli.HCatDriver} instead + */ +public class HCatDriver extends Driver { + + @Override + public CommandProcessorResponse run(String command) { + + CommandProcessorResponse cpr = null; + try { + cpr = super.run(command); + } catch (CommandNeedRetryException e) { + return new CommandProcessorResponse(-1, e.toString(), ""); + } + + SessionState ss = SessionState.get(); + + if (cpr.getResponseCode() == 0) { + // Only attempt to do this, if cmd was successful. + int rc = setFSPermsNGrp(ss); + cpr = new CommandProcessorResponse(rc); + } + // reset conf vars + ss.getConf().set(HCatConstants.HCAT_CREATE_DB_NAME, ""); + ss.getConf().set(HCatConstants.HCAT_CREATE_TBL_NAME, ""); + + return cpr; + } + + private int setFSPermsNGrp(SessionState ss) { + + Configuration conf = ss.getConf(); + + String tblName = conf.get(HCatConstants.HCAT_CREATE_TBL_NAME, ""); + if (tblName.isEmpty()) { + tblName = conf.get("import.destination.table", ""); + conf.set("import.destination.table", ""); + } + String dbName = conf.get(HCatConstants.HCAT_CREATE_DB_NAME, ""); + String grp = conf.get(HCatConstants.HCAT_GROUP, null); + String permsStr = conf.get(HCatConstants.HCAT_PERMS, null); + + if (tblName.isEmpty() && dbName.isEmpty()) { + // it wasn't create db/table + return 0; + } + + if (null == grp && null == permsStr) { + // there were no grp and perms to begin with. + return 0; + } + + FsPermission perms = FsPermission.valueOf(permsStr); + + if (!tblName.isEmpty()) { + Hive db = null; + try { + db = Hive.get(); + Table tbl = db.getTable(tblName); + Path tblPath = tbl.getPath(); + + FileSystem fs = tblPath.getFileSystem(conf); + if (null != perms) { + fs.setPermission(tblPath, perms); + } + if (null != grp) { + fs.setOwner(tblPath, null, grp); + } + return 0; + + } catch (Exception e) { + ss.err.println(String.format("Failed to set permissions/groups on TABLE: <%s> %s", tblName, e.getMessage())); + try { // We need to drop the table. + if (null != db) { + db.dropTable(tblName); + } + } catch (HiveException he) { + ss.err.println(String.format("Failed to drop TABLE <%s> after failing to set permissions/groups on it. %s", tblName, e.getMessage())); + } + return 1; + } + } else { + // looks like a db operation + if (dbName.isEmpty() || dbName.equals(MetaStoreUtils.DEFAULT_DATABASE_NAME)) { + // We dont set perms or groups for default dir. + return 0; + } else { + try { + Hive db = Hive.get(); + Path dbPath = new Warehouse(conf).getDatabasePath(db.getDatabase(dbName)); + FileSystem fs = dbPath.getFileSystem(conf); + if (perms != null) { + fs.setPermission(dbPath, perms); + } + if (null != grp) { + fs.setOwner(dbPath, null, grp); + } + return 0; + } catch (Exception e) { + ss.err.println(String.format("Failed to set permissions and/or group on DB: <%s> %s", dbName, e.getMessage())); + try { + Hive.get().dropDatabase(dbName); + } catch (Exception e1) { + ss.err.println(String.format("Failed to drop DB <%s> after failing to set permissions/group on it. %s", dbName, e1.getMessage())); + } + return 1; + } + } + } + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/cli/SemanticAnalysis/CreateDatabaseHook.java hcatalog/core/src/main/java/org/apache/hcatalog/cli/SemanticAnalysis/CreateDatabaseHook.java new file mode 100644 index 0000000..96d0d3d --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/cli/SemanticAnalysis/CreateDatabaseHook.java @@ -0,0 +1,99 @@ +/** + * 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.hcatalog.cli.SemanticAnalysis; + +import java.io.Serializable; +import java.util.List; + +import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.hadoop.hive.ql.exec.Task; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.parse.ASTNode; +import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer; +import org.apache.hadoop.hive.ql.parse.HiveParser; +import org.apache.hadoop.hive.ql.parse.HiveSemanticAnalyzerHookContext; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.plan.CreateDatabaseDesc; +import org.apache.hadoop.hive.ql.plan.DDLWork; +import org.apache.hadoop.hive.ql.security.authorization.Privilege; +import org.apache.hcatalog.common.HCatConstants; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.cli.SemanticAnalysis.CreateDatabaseHook} instead + */ +final class CreateDatabaseHook extends HCatSemanticAnalyzerBase { + + String databaseName; + + @Override + public ASTNode preAnalyze(HiveSemanticAnalyzerHookContext context, ASTNode ast) + throws SemanticException { + + Hive db; + try { + db = context.getHive(); + } catch (HiveException e) { + throw new SemanticException("Couldn't get Hive DB instance in semantic analysis phase.", e); + } + + // Analyze and create tbl properties object + int numCh = ast.getChildCount(); + + databaseName = BaseSemanticAnalyzer.getUnescapedName((ASTNode) ast.getChild(0)); + + for (int num = 1; num < numCh; num++) { + ASTNode child = (ASTNode) ast.getChild(num); + + switch (child.getToken().getType()) { + + case HiveParser.TOK_IFNOTEXISTS: + try { + List dbs = db.getDatabasesByPattern(databaseName); + if (dbs != null && dbs.size() > 0) { // db exists + return ast; + } + } catch (HiveException e) { + throw new SemanticException(e); + } + break; + } + } + + return ast; + } + + @Override + public void postAnalyze(HiveSemanticAnalyzerHookContext context, + List> rootTasks) throws SemanticException { + context.getConf().set(HCatConstants.HCAT_CREATE_DB_NAME, databaseName); + super.postAnalyze(context, rootTasks); + } + + @Override + protected void authorizeDDLWork(HiveSemanticAnalyzerHookContext context, + Hive hive, DDLWork work) throws HiveException { + CreateDatabaseDesc createDb = work.getCreateDatabaseDesc(); + if (createDb != null) { + Database db = new Database(createDb.getName(), createDb.getComment(), + createDb.getLocationUri(), createDb.getDatabaseProperties()); + authorize(db, Privilege.CREATE); + } + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/cli/SemanticAnalysis/CreateTableHook.java hcatalog/core/src/main/java/org/apache/hcatalog/cli/SemanticAnalysis/CreateTableHook.java new file mode 100644 index 0000000..f9b6497 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/cli/SemanticAnalysis/CreateTableHook.java @@ -0,0 +1,248 @@ +/** + * 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.hcatalog.cli.SemanticAnalysis; + +import java.io.IOException; +import java.io.Serializable; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.ql.exec.DDLTask; +import org.apache.hadoop.hive.ql.exec.Task; +import org.apache.hadoop.hive.ql.io.RCFileInputFormat; +import org.apache.hadoop.hive.ql.io.RCFileOutputFormat; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.parse.ASTNode; +import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer; +import org.apache.hadoop.hive.ql.parse.HiveParser; +import org.apache.hadoop.hive.ql.parse.HiveSemanticAnalyzerHookContext; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.plan.CreateTableDesc; +import org.apache.hadoop.hive.ql.security.authorization.Privilege; +import org.apache.hcatalog.common.HCatConstants; +import org.apache.hcatalog.common.HCatUtil; +import org.apache.hcatalog.mapreduce.HCatStorageHandler; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.cli.SemanticAnalysis.CreateTableHook} instead + */ +final class CreateTableHook extends HCatSemanticAnalyzerBase { + + private String tableName; + + @Override + public ASTNode preAnalyze(HiveSemanticAnalyzerHookContext context, + ASTNode ast) throws SemanticException { + + Hive db; + try { + db = context.getHive(); + } catch (HiveException e) { + throw new SemanticException( + "Couldn't get Hive DB instance in semantic analysis phase.", + e); + } + + // Analyze and create tbl properties object + int numCh = ast.getChildCount(); + + String inputFormat = null, outputFormat = null; + tableName = BaseSemanticAnalyzer.getUnescapedName((ASTNode) ast + .getChild(0)); + boolean likeTable = false; + + for (int num = 1; num < numCh; num++) { + ASTNode child = (ASTNode) ast.getChild(num); + + switch (child.getToken().getType()) { + + case HiveParser.TOK_QUERY: // CTAS + throw new SemanticException( + "Operation not supported. Create table as " + + "Select is not a valid operation."); + + case HiveParser.TOK_TABLEBUCKETS: + break; + + case HiveParser.TOK_TBLSEQUENCEFILE: + inputFormat = HCatConstants.SEQUENCEFILE_INPUT; + outputFormat = HCatConstants.SEQUENCEFILE_OUTPUT; + break; + + case HiveParser.TOK_TBLTEXTFILE: + inputFormat = org.apache.hadoop.mapred.TextInputFormat.class.getName(); + outputFormat = org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat.class.getName(); + + break; + + case HiveParser.TOK_LIKETABLE: + likeTable = true; + break; + + case HiveParser.TOK_IFNOTEXISTS: + try { + List tables = db.getTablesByPattern(tableName); + if (tables != null && tables.size() > 0) { // table + // exists + return ast; + } + } catch (HiveException e) { + throw new SemanticException(e); + } + break; + + case HiveParser.TOK_TABLEPARTCOLS: + List partCols = BaseSemanticAnalyzer + .getColumns((ASTNode) child.getChild(0), false); + for (FieldSchema fs : partCols) { + if (!fs.getType().equalsIgnoreCase("string")) { + throw new SemanticException( + "Operation not supported. HCatalog only " + + "supports partition columns of type string. " + + "For column: " + + fs.getName() + + " Found type: " + fs.getType()); + } + } + break; + + case HiveParser.TOK_STORAGEHANDLER: + String storageHandler = BaseSemanticAnalyzer + .unescapeSQLString(child.getChild(0).getText()); + if (org.apache.commons.lang.StringUtils + .isNotEmpty(storageHandler)) { + return ast; + } + + break; + + case HiveParser.TOK_TABLEFILEFORMAT: + if (child.getChildCount() < 2) { + throw new SemanticException( + "Incomplete specification of File Format. " + + "You must provide InputFormat, OutputFormat."); + } + inputFormat = BaseSemanticAnalyzer.unescapeSQLString(child + .getChild(0).getText()); + outputFormat = BaseSemanticAnalyzer.unescapeSQLString(child + .getChild(1).getText()); + break; + + case HiveParser.TOK_TBLRCFILE: + inputFormat = RCFileInputFormat.class.getName(); + outputFormat = RCFileOutputFormat.class.getName(); + break; + + } + } + + if (!likeTable && (inputFormat == null || outputFormat == null)) { + throw new SemanticException( + "STORED AS specification is either incomplete or incorrect."); + } + + + return ast; + } + + @Override + public void postAnalyze(HiveSemanticAnalyzerHookContext context, + List> rootTasks) + throws SemanticException { + + if (rootTasks.size() == 0) { + // There will be no DDL task created in case if its CREATE TABLE IF + // NOT EXISTS + return; + } + CreateTableDesc desc = ((DDLTask) rootTasks.get(rootTasks.size() - 1)) + .getWork().getCreateTblDesc(); + if (desc == null) { + // Desc will be null if its CREATE TABLE LIKE. Desc will be + // contained in CreateTableLikeDesc. Currently, HCat disallows CTLT in + // pre-hook. So, desc can never be null. + return; + } + Map tblProps = desc.getTblProps(); + if (tblProps == null) { + // tblProps will be null if user didnt use tblprops in his CREATE + // TABLE cmd. + tblProps = new HashMap(); + + } + + // first check if we will allow the user to create table. + String storageHandler = desc.getStorageHandler(); + if (StringUtils.isEmpty(storageHandler)) { + } else { + try { + HCatStorageHandler storageHandlerInst = HCatUtil + .getStorageHandler(context.getConf(), + desc.getStorageHandler(), + desc.getSerName(), + desc.getInputFormat(), + desc.getOutputFormat()); + //Authorization checks are performed by the storageHandler.getAuthorizationProvider(), if + //StorageDelegationAuthorizationProvider is used. + } catch (IOException e) { + throw new SemanticException(e); + } + } + + if (desc != null) { + try { + Table table = context.getHive().newTable(desc.getTableName()); + if (desc.getLocation() != null) { + table.setDataLocation(new Path(desc.getLocation()).toUri()); + } + if (desc.getStorageHandler() != null) { + table.setProperty( + org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_STORAGE, + desc.getStorageHandler()); + } + for (Map.Entry prop : tblProps.entrySet()) { + table.setProperty(prop.getKey(), prop.getValue()); + } + for (Map.Entry prop : desc.getSerdeProps().entrySet()) { + table.setSerdeParam(prop.getKey(), prop.getValue()); + } + //TODO: set other Table properties as needed + + //authorize against the table operation so that location permissions can be checked if any + + if (HiveConf.getBoolVar(context.getConf(), + HiveConf.ConfVars.HIVE_AUTHORIZATION_ENABLED)) { + authorize(table, Privilege.CREATE); + } + } catch (HiveException ex) { + throw new SemanticException(ex); + } + } + + desc.setTblProps(tblProps); + context.getConf().set(HCatConstants.HCAT_CREATE_TBL_NAME, tableName); + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzer.java hcatalog/core/src/main/java/org/apache/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzer.java new file mode 100644 index 0000000..13d8a0c --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzer.java @@ -0,0 +1,378 @@ +/** + * 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.hcatalog.cli.SemanticAnalysis; + +import java.io.Serializable; +import java.util.List; + +import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.hadoop.hive.ql.exec.Task; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.metadata.Partition; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.parse.ASTNode; +import org.apache.hadoop.hive.ql.parse.AbstractSemanticAnalyzerHook; +import org.apache.hadoop.hive.ql.parse.HiveParser; +import org.apache.hadoop.hive.ql.parse.HiveSemanticAnalyzerHookContext; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.plan.AlterTableDesc; +import org.apache.hadoop.hive.ql.plan.DDLWork; +import org.apache.hadoop.hive.ql.plan.DescDatabaseDesc; +import org.apache.hadoop.hive.ql.plan.DescTableDesc; +import org.apache.hadoop.hive.ql.plan.DropDatabaseDesc; +import org.apache.hadoop.hive.ql.plan.DropTableDesc; +import org.apache.hadoop.hive.ql.plan.HiveOperation; +import org.apache.hadoop.hive.ql.plan.PartitionSpec; +import org.apache.hadoop.hive.ql.plan.ShowDatabasesDesc; +import org.apache.hadoop.hive.ql.plan.ShowPartitionsDesc; +import org.apache.hadoop.hive.ql.plan.ShowTableStatusDesc; +import org.apache.hadoop.hive.ql.plan.ShowTablesDesc; +import org.apache.hadoop.hive.ql.plan.SwitchDatabaseDesc; +import org.apache.hadoop.hive.ql.security.authorization.Privilege; +import org.apache.hcatalog.common.ErrorType; +import org.apache.hcatalog.common.HCatException; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.cli.SemanticAnalysis.HCatSemanticAnalyzer} instead + */ +public class HCatSemanticAnalyzer extends HCatSemanticAnalyzerBase { + + private AbstractSemanticAnalyzerHook hook; + private ASTNode ast; + + + @Override + public ASTNode preAnalyze(HiveSemanticAnalyzerHookContext context, ASTNode ast) + throws SemanticException { + + this.ast = ast; + switch (ast.getToken().getType()) { + + // HCat wants to intercept following tokens and special-handle them. + case HiveParser.TOK_CREATETABLE: + hook = new CreateTableHook(); + return hook.preAnalyze(context, ast); + + case HiveParser.TOK_CREATEDATABASE: + hook = new CreateDatabaseHook(); + return hook.preAnalyze(context, ast); + + case HiveParser.TOK_ALTERTABLE_PARTITION: + if (((ASTNode) ast.getChild(1)).getToken().getType() == HiveParser.TOK_ALTERTABLE_FILEFORMAT) { + return ast; + } else if (((ASTNode) ast.getChild(1)).getToken().getType() == HiveParser.TOK_ALTERTABLE_ALTERPARTS_MERGEFILES) { + // unsupported + throw new SemanticException("Operation not supported."); + } else { + return ast; + } + + // HCat will allow these operations to be performed. + // Database DDL + case HiveParser.TOK_SHOWDATABASES: + case HiveParser.TOK_DROPDATABASE: + case HiveParser.TOK_SWITCHDATABASE: + case HiveParser.TOK_DESCDATABASE: + case HiveParser.TOK_ALTERDATABASE_PROPERTIES: + + // Index DDL + case HiveParser.TOK_ALTERINDEX_PROPERTIES: + case HiveParser.TOK_CREATEINDEX: + case HiveParser.TOK_DROPINDEX: + case HiveParser.TOK_SHOWINDEXES: + + // View DDL + // "alter view add partition" does not work because of the nature of implementation + // of the DDL in hive. Hive will internally invoke another Driver on the select statement, + // and HCat does not let "select" statement through. I cannot find a way to get around it + // without modifying hive code. So just leave it unsupported. + //case HiveParser.TOK_ALTERVIEW_ADDPARTS: + case HiveParser.TOK_ALTERVIEW_DROPPARTS: + case HiveParser.TOK_ALTERVIEW_PROPERTIES: + case HiveParser.TOK_ALTERVIEW_RENAME: + case HiveParser.TOK_CREATEVIEW: + case HiveParser.TOK_DROPVIEW: + + // Authorization DDL + case HiveParser.TOK_CREATEROLE: + case HiveParser.TOK_DROPROLE: + case HiveParser.TOK_GRANT_ROLE: + case HiveParser.TOK_GRANT_WITH_OPTION: + case HiveParser.TOK_GRANT: + case HiveParser.TOK_REVOKE_ROLE: + case HiveParser.TOK_REVOKE: + case HiveParser.TOK_SHOW_GRANT: + case HiveParser.TOK_SHOW_ROLE_GRANT: + + // Misc DDL + case HiveParser.TOK_LOCKTABLE: + case HiveParser.TOK_UNLOCKTABLE: + case HiveParser.TOK_SHOWLOCKS: + case HiveParser.TOK_DESCFUNCTION: + case HiveParser.TOK_SHOWFUNCTIONS: + case HiveParser.TOK_EXPLAIN: + + // Table DDL + case HiveParser.TOK_ALTERTABLE_ADDPARTS: + case HiveParser.TOK_ALTERTABLE_ADDCOLS: + case HiveParser.TOK_ALTERTABLE_CHANGECOL_AFTER_POSITION: + case HiveParser.TOK_ALTERTABLE_SERDEPROPERTIES: + case HiveParser.TOK_ALTERTABLE_CLUSTER_SORT: + case HiveParser.TOK_ALTERTABLE_DROPPARTS: + case HiveParser.TOK_ALTERTABLE_PROPERTIES: + case HiveParser.TOK_ALTERTABLE_RENAME: + case HiveParser.TOK_ALTERTABLE_RENAMECOL: + case HiveParser.TOK_ALTERTABLE_REPLACECOLS: + case HiveParser.TOK_ALTERTABLE_SERIALIZER: + case HiveParser.TOK_ALTERTABLE_TOUCH: + case HiveParser.TOK_DESCTABLE: + case HiveParser.TOK_DROPTABLE: + case HiveParser.TOK_SHOW_TABLESTATUS: + case HiveParser.TOK_SHOWPARTITIONS: + case HiveParser.TOK_SHOWTABLES: + return ast; + + // In all other cases, throw an exception. Its a white-list of allowed operations. + default: + throw new SemanticException("Operation not supported."); + + } + } + + @Override + public void postAnalyze(HiveSemanticAnalyzerHookContext context, + List> rootTasks) throws SemanticException { + + try { + + switch (ast.getToken().getType()) { + + case HiveParser.TOK_CREATETABLE: + case HiveParser.TOK_CREATEDATABASE: + case HiveParser.TOK_ALTERTABLE_PARTITION: + + // HCat will allow these operations to be performed. + // Database DDL + case HiveParser.TOK_SHOWDATABASES: + case HiveParser.TOK_DROPDATABASE: + case HiveParser.TOK_SWITCHDATABASE: + case HiveParser.TOK_DESCDATABASE: + case HiveParser.TOK_ALTERDATABASE_PROPERTIES: + + // Index DDL + case HiveParser.TOK_ALTERINDEX_PROPERTIES: + case HiveParser.TOK_CREATEINDEX: + case HiveParser.TOK_DROPINDEX: + case HiveParser.TOK_SHOWINDEXES: + + // View DDL + //case HiveParser.TOK_ALTERVIEW_ADDPARTS: + case HiveParser.TOK_ALTERVIEW_DROPPARTS: + case HiveParser.TOK_ALTERVIEW_PROPERTIES: + case HiveParser.TOK_ALTERVIEW_RENAME: + case HiveParser.TOK_CREATEVIEW: + case HiveParser.TOK_DROPVIEW: + + // Authorization DDL + case HiveParser.TOK_CREATEROLE: + case HiveParser.TOK_DROPROLE: + case HiveParser.TOK_GRANT_ROLE: + case HiveParser.TOK_GRANT_WITH_OPTION: + case HiveParser.TOK_GRANT: + case HiveParser.TOK_REVOKE_ROLE: + case HiveParser.TOK_REVOKE: + case HiveParser.TOK_SHOW_GRANT: + case HiveParser.TOK_SHOW_ROLE_GRANT: + + // Misc DDL + case HiveParser.TOK_LOCKTABLE: + case HiveParser.TOK_UNLOCKTABLE: + case HiveParser.TOK_SHOWLOCKS: + case HiveParser.TOK_DESCFUNCTION: + case HiveParser.TOK_SHOWFUNCTIONS: + case HiveParser.TOK_EXPLAIN: + + // Table DDL + case HiveParser.TOK_ALTERTABLE_ADDPARTS: + case HiveParser.TOK_ALTERTABLE_ADDCOLS: + case HiveParser.TOK_ALTERTABLE_CHANGECOL_AFTER_POSITION: + case HiveParser.TOK_ALTERTABLE_SERDEPROPERTIES: + case HiveParser.TOK_ALTERTABLE_CLUSTER_SORT: + case HiveParser.TOK_ALTERTABLE_DROPPARTS: + case HiveParser.TOK_ALTERTABLE_PROPERTIES: + case HiveParser.TOK_ALTERTABLE_RENAME: + case HiveParser.TOK_ALTERTABLE_RENAMECOL: + case HiveParser.TOK_ALTERTABLE_REPLACECOLS: + case HiveParser.TOK_ALTERTABLE_SERIALIZER: + case HiveParser.TOK_ALTERTABLE_TOUCH: + case HiveParser.TOK_DESCTABLE: + case HiveParser.TOK_DROPTABLE: + case HiveParser.TOK_SHOW_TABLESTATUS: + case HiveParser.TOK_SHOWPARTITIONS: + case HiveParser.TOK_SHOWTABLES: + break; + + default: + throw new HCatException(ErrorType.ERROR_INTERNAL_EXCEPTION, "Unexpected token: " + ast.getToken()); + } + + authorizeDDL(context, rootTasks); + + } catch (HCatException e) { + throw new SemanticException(e); + } catch (HiveException e) { + throw new SemanticException(e); + } + + if (hook != null) { + hook.postAnalyze(context, rootTasks); + } + } + + private String extractTableName(String compoundName) { + /* + * the table name can potentially be a dot-format one with column names + * specified as part of the table name. e.g. a.b.c where b is a column in + * a and c is a field of the object/column b etc. For authorization + * purposes, we should use only the first part of the dotted name format. + * + */ + + String[] words = compoundName.split("\\."); + return words[0]; + } + + @Override + protected void authorizeDDLWork(HiveSemanticAnalyzerHookContext cntxt, Hive hive, DDLWork work) + throws HiveException { + // DB opereations, none of them are enforced by Hive right now. + + ShowDatabasesDesc showDatabases = work.getShowDatabasesDesc(); + if (showDatabases != null) { + authorize(HiveOperation.SHOWDATABASES.getInputRequiredPrivileges(), + HiveOperation.SHOWDATABASES.getOutputRequiredPrivileges()); + } + + DropDatabaseDesc dropDb = work.getDropDatabaseDesc(); + if (dropDb != null) { + Database db = cntxt.getHive().getDatabase(dropDb.getDatabaseName()); + authorize(db, Privilege.DROP); + } + + DescDatabaseDesc descDb = work.getDescDatabaseDesc(); + if (descDb != null) { + Database db = cntxt.getHive().getDatabase(descDb.getDatabaseName()); + authorize(db, Privilege.SELECT); + } + + SwitchDatabaseDesc switchDb = work.getSwitchDatabaseDesc(); + if (switchDb != null) { + Database db = cntxt.getHive().getDatabase(switchDb.getDatabaseName()); + authorize(db, Privilege.SELECT); + } + + ShowTablesDesc showTables = work.getShowTblsDesc(); + if (showTables != null) { + String dbName = showTables.getDbName() == null ? cntxt.getHive().getCurrentDatabase() + : showTables.getDbName(); + authorize(cntxt.getHive().getDatabase(dbName), Privilege.SELECT); + } + + ShowTableStatusDesc showTableStatus = work.getShowTblStatusDesc(); + if (showTableStatus != null) { + String dbName = showTableStatus.getDbName() == null ? cntxt.getHive().getCurrentDatabase() + : showTableStatus.getDbName(); + authorize(cntxt.getHive().getDatabase(dbName), Privilege.SELECT); + } + + // TODO: add alter database support in HCat + + // Table operations. + + DropTableDesc dropTable = work.getDropTblDesc(); + if (dropTable != null) { + if (dropTable.getPartSpecs() == null) { + // drop table is already enforced by Hive. We only check for table level location even if the + // table is partitioned. + } else { + //this is actually a ALTER TABLE DROP PARITITION statement + for (PartitionSpec partSpec : dropTable.getPartSpecs()) { + // partitions are not added as write entries in drop partitions in Hive + Table table = hive.getTable(hive.getCurrentDatabase(), dropTable.getTableName()); + List partitions = null; + try { + partitions = hive.getPartitionsByFilter(table, partSpec.toString()); + } catch (Exception e) { + throw new HiveException(e); + } + + for (Partition part : partitions) { + authorize(part, Privilege.DROP); + } + } + } + } + + AlterTableDesc alterTable = work.getAlterTblDesc(); + if (alterTable != null) { + Table table = hive.getTable(hive.getCurrentDatabase(), alterTable.getOldName(), false); + + Partition part = null; + if (alterTable.getPartSpec() != null) { + part = hive.getPartition(table, alterTable.getPartSpec(), false); + } + + String newLocation = alterTable.getNewLocation(); + + /* Hcat requires ALTER_DATA privileges for ALTER TABLE LOCATION statements + * for the old table/partition location and the new location. + */ + if (alterTable.getOp() == AlterTableDesc.AlterTableTypes.ALTERLOCATION) { + if (part != null) { + authorize(part, Privilege.ALTER_DATA); // authorize for the old + // location, and new location + part.setLocation(newLocation); + authorize(part, Privilege.ALTER_DATA); + } else { + authorize(table, Privilege.ALTER_DATA); // authorize for the old + // location, and new location + table.getTTable().getSd().setLocation(newLocation); + authorize(table, Privilege.ALTER_DATA); + } + } + //other alter operations are already supported by Hive + } + + // we should be careful when authorizing table based on just the + // table name. If columns have separate authorization domain, it + // must be honored + DescTableDesc descTable = work.getDescTblDesc(); + if (descTable != null) { + String tableName = extractTableName(descTable.getTableName()); + authorizeTable(cntxt.getHive(), tableName, Privilege.SELECT); + } + + ShowPartitionsDesc showParts = work.getShowPartsDesc(); + if (showParts != null) { + String tableName = extractTableName(showParts.getTabName()); + authorizeTable(cntxt.getHive(), tableName, Privilege.SELECT); + } + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzerBase.java hcatalog/core/src/main/java/org/apache/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzerBase.java new file mode 100644 index 0000000..6b78800 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/cli/SemanticAnalysis/HCatSemanticAnalyzerBase.java @@ -0,0 +1,180 @@ +/** + * 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.hcatalog.cli.SemanticAnalysis; + +import java.io.Serializable; +import java.util.List; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.hadoop.hive.ql.exec.Task; +import org.apache.hadoop.hive.ql.metadata.AuthorizationException; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.metadata.InvalidTableException; +import org.apache.hadoop.hive.ql.metadata.Partition; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.parse.AbstractSemanticAnalyzerHook; +import org.apache.hadoop.hive.ql.parse.HiveSemanticAnalyzerHookContext; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.plan.DDLWork; +import org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider; +import org.apache.hadoop.hive.ql.security.authorization.Privilege; +import org.apache.hadoop.hive.ql.session.SessionState; + +/** + * Base class for HCatSemanticAnalyzer hooks. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.cli.SemanticAnalysis.HCatSemanticAnalyzerBase} instead + */ +public class HCatSemanticAnalyzerBase extends AbstractSemanticAnalyzerHook { + + private HiveAuthorizationProvider authProvider; + + protected String getDbName(Hive hive, String dbName) { + return dbName == null ? hive.getCurrentDatabase() : dbName; + } + + public HiveAuthorizationProvider getAuthProvider() { + if (authProvider == null) { + authProvider = SessionState.get().getAuthorizer(); + } + + return authProvider; + } + + @Override + public void postAnalyze(HiveSemanticAnalyzerHookContext context, + List> rootTasks) throws SemanticException { + super.postAnalyze(context, rootTasks); + + //Authorize the operation. + authorizeDDL(context, rootTasks); + } + + /** + * Checks for the given rootTasks, and calls authorizeDDLWork() for each DDLWork to + * be authorized. The hooks should override this, or authorizeDDLWork to perform the + * actual authorization. + */ + /* + * Impl note: Hive provides authorization with it's own model, and calls the defined + * HiveAuthorizationProvider from Driver.doAuthorization(). However, HCat has to + * do additional calls to the auth provider to implement expected behavior for + * StorageDelegationAuthorizationProvider. This means, that the defined auth provider + * is called by both Hive and HCat. The following are missing from Hive's implementation, + * and when they are fixed in Hive, we can remove the HCat-specific auth checks. + * 1. CREATE DATABASE/TABLE, ADD PARTITION statements does not call + * HiveAuthorizationProvider.authorize() with the candidate objects, which means that + * we cannot do checks against defined LOCATION. + * 2. HiveOperation does not define sufficient Privileges for most of the operations, + * especially database operations. + * 3. For some of the operations, Hive SemanticAnalyzer does not add the changed + * object as a WriteEntity or ReadEntity. + * + * @see https://issues.apache.org/jira/browse/HCATALOG-244 + * @see https://issues.apache.org/jira/browse/HCATALOG-245 + */ + protected void authorizeDDL(HiveSemanticAnalyzerHookContext context, + List> rootTasks) throws SemanticException { + + if (!HiveConf.getBoolVar(context.getConf(), + HiveConf.ConfVars.HIVE_AUTHORIZATION_ENABLED)) { + return; + } + + Hive hive; + try { + hive = context.getHive(); + + for (Task task : rootTasks) { + if (task.getWork() instanceof DDLWork) { + DDLWork work = (DDLWork) task.getWork(); + if (work != null) { + authorizeDDLWork(context, hive, work); + } + } + } + } catch (SemanticException ex) { + throw ex; + } catch (AuthorizationException ex) { + throw ex; + } catch (Exception ex) { + throw new SemanticException(ex); + } + } + + /** + * Authorized the given DDLWork. Does nothing by default. Override this + * and delegate to the relevant method in HiveAuthorizationProvider obtained by + * getAuthProvider(). + */ + protected void authorizeDDLWork(HiveSemanticAnalyzerHookContext context, + Hive hive, DDLWork work) throws HiveException { + } + + protected void authorize(Privilege[] inputPrivs, Privilege[] outputPrivs) + throws AuthorizationException, SemanticException { + try { + getAuthProvider().authorize(inputPrivs, outputPrivs); + } catch (HiveException ex) { + throw new SemanticException(ex); + } + } + + protected void authorize(Database db, Privilege priv) + throws AuthorizationException, SemanticException { + try { + getAuthProvider().authorize(db, null, new Privilege[]{priv}); + } catch (HiveException ex) { + throw new SemanticException(ex); + } + } + + protected void authorizeTable(Hive hive, String tableName, Privilege priv) + throws AuthorizationException, HiveException { + Table table; + try { + table = hive.getTable(tableName); + } catch (InvalidTableException ite) { + // Table itself doesn't exist in metastore, nothing to validate. + return; + } + + authorize(table, priv); + } + + protected void authorize(Table table, Privilege priv) + throws AuthorizationException, SemanticException { + try { + getAuthProvider().authorize(table, new Privilege[]{priv}, null); + } catch (HiveException ex) { + throw new SemanticException(ex); + } + } + + protected void authorize(Partition part, Privilege priv) + throws AuthorizationException, SemanticException { + try { + getAuthProvider().authorize(part, new Privilege[]{priv}, null); + } catch (HiveException ex) { + throw new SemanticException(ex); + } + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/common/ErrorType.java hcatalog/core/src/main/java/org/apache/hcatalog/common/ErrorType.java new file mode 100644 index 0000000..88f9c11 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/common/ErrorType.java @@ -0,0 +1,138 @@ +/** + * 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.hcatalog.common; + +/** + * Enum type representing the various errors throws by HCat. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.common.ErrorType} instead + */ +public enum ErrorType { + + /* HCat Input Format related errors 1000 - 1999 */ + ERROR_DB_INIT (1000, "Error initializing database session"), + ERROR_EXCEED_MAXPART (1001, "Query result exceeded maximum number of partitions allowed"), + + ERROR_SET_INPUT (1002, "Error setting input information"), + + /* HCat Output Format related errors 2000 - 2999 */ + ERROR_INVALID_TABLE (2000, "Table specified does not exist"), + ERROR_SET_OUTPUT (2001, "Error setting output information"), + ERROR_DUPLICATE_PARTITION (2002, "Partition already present with given partition key values"), + ERROR_NON_EMPTY_TABLE (2003, "Non-partitioned table already contains data"), + ERROR_NOT_INITIALIZED (2004, "HCatOutputFormat not initialized, setOutput has to be called"), + ERROR_INIT_STORAGE_HANDLER (2005, "Error initializing storage handler instance"), + ERROR_PUBLISHING_PARTITION (2006, "Error adding partition to metastore"), + ERROR_SCHEMA_COLUMN_MISMATCH (2007, "Invalid column position in partition schema"), + ERROR_SCHEMA_PARTITION_KEY (2008, "Partition key cannot be present in the partition data"), + ERROR_SCHEMA_TYPE_MISMATCH (2009, "Invalid column type in partition schema"), + ERROR_INVALID_PARTITION_VALUES (2010, "Invalid partition values specified"), + ERROR_MISSING_PARTITION_KEY (2011, "Partition key value not provided for publish"), + ERROR_MOVE_FAILED (2012, "Moving of data failed during commit"), + ERROR_TOO_MANY_DYNAMIC_PTNS (2013, "Attempt to create too many dynamic partitions"), + ERROR_INIT_LOADER (2014, "Error initializing Pig loader"), + ERROR_INIT_STORER (2015, "Error initializing Pig storer"), + ERROR_NOT_SUPPORTED (2016, "Error operation not supported"), + + /* Authorization Errors 3000 - 3999 */ + ERROR_ACCESS_CONTROL (3000, "Permission denied"), + + /* Miscellaneous errors, range 9000 - 9998 */ + ERROR_UNIMPLEMENTED (9000, "Functionality currently unimplemented"), + ERROR_INTERNAL_EXCEPTION (9001, "Exception occurred while processing HCat request"); + + /** The error code. */ + private int errorCode; + + /** The error message. */ + private String errorMessage; + + /** Should the causal exception message be appended to the error message, yes by default*/ + private boolean appendCauseMessage = true; + + /** Is this a retriable error, no by default. */ + private boolean isRetriable = false; + + /** + * Instantiates a new error type. + * @param errorCode the error code + * @param errorMessage the error message + */ + private ErrorType(int errorCode, String errorMessage) { + this.errorCode = errorCode; + this.errorMessage = errorMessage; + } + + /** + * Instantiates a new error type. + * @param errorCode the error code + * @param errorMessage the error message + * @param appendCauseMessage should causal exception message be appended to error message + */ + private ErrorType(int errorCode, String errorMessage, boolean appendCauseMessage) { + this.errorCode = errorCode; + this.errorMessage = errorMessage; + this.appendCauseMessage = appendCauseMessage; + } + + /** + * Instantiates a new error type. + * @param errorCode the error code + * @param errorMessage the error message + * @param appendCauseMessage should causal exception message be appended to error message + * @param isRetriable is this a retriable error + */ + private ErrorType(int errorCode, String errorMessage, boolean appendCauseMessage, boolean isRetriable) { + this.errorCode = errorCode; + this.errorMessage = errorMessage; + this.appendCauseMessage = appendCauseMessage; + this.isRetriable = isRetriable; + } + + /** + * Gets the error code. + * @return the error code + */ + public int getErrorCode() { + return errorCode; + } + + /** + * Gets the error message. + * @return the error message + */ + public String getErrorMessage() { + return errorMessage; + } + + /** + * Checks if this is a retriable error. + * @return true, if is a retriable error, false otherwise + */ + public boolean isRetriable() { + return isRetriable; + } + + /** + * Whether the cause of the exception should be added to the error message. + * @return true, if the cause should be added to the message, false otherwise + */ + public boolean appendCauseMessage() { + return appendCauseMessage; + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/common/HCatConstants.java hcatalog/core/src/main/java/org/apache/hcatalog/common/HCatConstants.java new file mode 100644 index 0000000..e1e04c8 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/common/HCatConstants.java @@ -0,0 +1,189 @@ +/** + * 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.hcatalog.common; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.mapred.SequenceFileInputFormat; +import org.apache.hadoop.mapred.SequenceFileOutputFormat; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.common.HCatConstants} instead + */ +public final class HCatConstants { + + public static final String HIVE_RCFILE_IF_CLASS = "org.apache.hadoop.hive.ql.io.RCFileInputFormat"; + public static final String HIVE_RCFILE_OF_CLASS = "org.apache.hadoop.hive.ql.io.RCFileOutputFormat"; + + public static final String SEQUENCEFILE_INPUT = SequenceFileInputFormat.class.getName(); + public static final String SEQUENCEFILE_OUTPUT = SequenceFileOutputFormat.class.getName(); + + public static final String HCAT_PIG_STORAGE_CLASS = "org.apache.pig.builtin.PigStorage"; + public static final String HCAT_PIG_LOADER = "hcat.pig.loader"; + public static final String HCAT_PIG_LOADER_LOCATION_SET = HCAT_PIG_LOADER + ".location.set"; + public static final String HCAT_PIG_LOADER_ARGS = "hcat.pig.loader.args"; + public static final String HCAT_PIG_STORER = "hcat.pig.storer"; + public static final String HCAT_PIG_STORER_ARGS = "hcat.pig.storer.args"; + public static final String HCAT_PIG_ARGS_DELIMIT = "hcat.pig.args.delimiter"; + public static final String HCAT_PIG_ARGS_DELIMIT_DEFAULT = ","; + public static final String HCAT_PIG_STORER_LOCATION_SET = HCAT_PIG_STORER + ".location.set"; + public static final String HCAT_PIG_INNER_TUPLE_NAME = "hcat.pig.inner.tuple.name"; + public static final String HCAT_PIG_INNER_TUPLE_NAME_DEFAULT = "innertuple"; + public static final String HCAT_PIG_INNER_FIELD_NAME = "hcat.pig.inner.field.name"; + public static final String HCAT_PIG_INNER_FIELD_NAME_DEFAULT = "innerfield"; + + /** + * {@value} (default: null) + * When the property is set in the UDFContext of the org.apache.hcatalog.pig.HCatStorer, HCatStorer writes + * to the location it specifies instead of the default HCatalog location format. An example can be found + * in org.apache.hcatalog.pig.HCatStorerWrapper. + */ + public static final String HCAT_PIG_STORER_EXTERNAL_LOCATION = HCAT_PIG_STORER + ".external.location"; + + //The keys used to store info into the job Configuration + public static final String HCAT_KEY_BASE = "mapreduce.lib.hcat"; + + public static final String HCAT_KEY_OUTPUT_SCHEMA = HCAT_KEY_BASE + ".output.schema"; + + public static final String HCAT_KEY_JOB_INFO = HCAT_KEY_BASE + ".job.info"; + + // hcatalog specific configurations, that can be put in hive-site.xml + public static final String HCAT_HIVE_CLIENT_EXPIRY_TIME = "hcatalog.hive.client.cache.expiry.time"; + + private HCatConstants() { // restrict instantiation + } + + public static final String HCAT_TABLE_SCHEMA = "hcat.table.schema"; + + public static final String HCAT_METASTORE_URI = HiveConf.ConfVars.METASTOREURIS.varname; + + public static final String HCAT_PERMS = "hcat.perms"; + + public static final String HCAT_GROUP = "hcat.group"; + + public static final String HCAT_CREATE_TBL_NAME = "hcat.create.tbl.name"; + + public static final String HCAT_CREATE_DB_NAME = "hcat.create.db.name"; + + public static final String HCAT_METASTORE_PRINCIPAL + = HiveConf.ConfVars.METASTORE_KERBEROS_PRINCIPAL.varname; + + /** + * The desired number of input splits produced for each partition. When the + * input files are large and few, we want to split them into many splits, + * so as to increase the parallelizm of loading the splits. Try also two + * other parameters, mapred.min.split.size and mapred.max.split.size, to + * control the number of input splits. + */ + public static final String HCAT_DESIRED_PARTITION_NUM_SPLITS = + "hcat.desired.partition.num.splits"; + + // IMPORTANT IMPORTANT IMPORTANT!!!!! + //The keys used to store info into the job Configuration. + //If any new keys are added, the HCatStorer needs to be updated. The HCatStorer + //updates the job configuration in the backend to insert these keys to avoid + //having to call setOutput from the backend (which would cause a metastore call + //from the map jobs) + public static final String HCAT_KEY_OUTPUT_BASE = "mapreduce.lib.hcatoutput"; + public static final String HCAT_KEY_OUTPUT_INFO = HCAT_KEY_OUTPUT_BASE + ".info"; + public static final String HCAT_KEY_HIVE_CONF = HCAT_KEY_OUTPUT_BASE + ".hive.conf"; + public static final String HCAT_KEY_TOKEN_SIGNATURE = HCAT_KEY_OUTPUT_BASE + ".token.sig"; + + public static final String[] OUTPUT_CONFS_TO_SAVE = { + HCAT_KEY_OUTPUT_INFO, + HCAT_KEY_HIVE_CONF, + HCAT_KEY_TOKEN_SIGNATURE + }; + + + public static final String HCAT_MSG_CLEAN_FREQ = "hcat.msg.clean.freq"; + public static final String HCAT_MSG_EXPIRY_DURATION = "hcat.msg.expiry.duration"; + + public static final String HCAT_MSGBUS_TOPIC_NAME = "hcat.msgbus.topic.name"; + public static final String HCAT_MSGBUS_TOPIC_NAMING_POLICY = "hcat.msgbus.topic.naming.policy"; + public static final String HCAT_MSGBUS_TOPIC_PREFIX = "hcat.msgbus.topic.prefix"; + + public static final String HCAT_DYNAMIC_PTN_JOBID = HCAT_KEY_OUTPUT_BASE + "dynamic.jobid"; + public static final boolean HCAT_IS_DYNAMIC_MAX_PTN_CHECK_ENABLED = false; + + // Message Bus related properties. + public static final String HCAT_DEFAULT_TOPIC_PREFIX = "hcat"; + public static final String HCAT_EVENT = "HCAT_EVENT"; + public static final String HCAT_ADD_PARTITION_EVENT = "ADD_PARTITION"; + public static final String HCAT_DROP_PARTITION_EVENT = "DROP_PARTITION"; + public static final String HCAT_PARTITION_DONE_EVENT = "PARTITION_DONE"; + public static final String HCAT_CREATE_TABLE_EVENT = "CREATE_TABLE"; + public static final String HCAT_DROP_TABLE_EVENT = "DROP_TABLE"; + public static final String HCAT_CREATE_DATABASE_EVENT = "CREATE_DATABASE"; + public static final String HCAT_DROP_DATABASE_EVENT = "DROP_DATABASE"; + public static final String HCAT_MESSAGE_VERSION = "HCAT_MESSAGE_VERSION"; + public static final String HCAT_MESSAGE_FORMAT = "HCAT_MESSAGE_FORMAT"; + public static final String CONF_LABEL_HCAT_MESSAGE_FACTORY_IMPL_PREFIX = "hcatalog.message.factory.impl."; + public static final String CONF_LABEL_HCAT_MESSAGE_FORMAT = "hcatalog.message.format"; + public static final String DEFAULT_MESSAGE_FACTORY_IMPL = "org.apache.hcatalog.messaging.json.JSONMessageFactory"; + + // System environment variables + public static final String SYSENV_HADOOP_TOKEN_FILE_LOCATION = "HADOOP_TOKEN_FILE_LOCATION"; + + // Hadoop Conf Var Names + public static final String CONF_MAPREDUCE_JOB_CREDENTIALS_BINARY = "mapreduce.job.credentials.binary"; + + //*************************************************************************** + // Data-related configuration properties. + //*************************************************************************** + + /** + * {@value} (default: {@value #HCAT_DATA_CONVERT_BOOLEAN_TO_INTEGER_DEFAULT}). + * Pig < 0.10.0 does not have boolean support, and scripts written for pre-boolean Pig versions + * will not expect boolean values when upgrading Pig. For integration the option is offered to + * convert boolean fields to integers by setting this Hadoop configuration key. + */ + public static final String HCAT_DATA_CONVERT_BOOLEAN_TO_INTEGER = + "hcat.data.convert.boolean.to.integer"; + public static final boolean HCAT_DATA_CONVERT_BOOLEAN_TO_INTEGER_DEFAULT = false; + + /** + * {@value} (default: {@value #HCAT_DATA_TINY_SMALL_INT_PROMOTION_DEFAULT}). + * Hive tables support tinyint and smallint columns, while not all processing frameworks support + * these types (Pig only has integer for example). Enable this property to promote tinyint and + * smallint columns to integer at runtime. Note that writes to tinyint and smallint columns + * enforce bounds checking and jobs will fail if attempting to write values outside the column + * bounds. + */ + public static final String HCAT_DATA_TINY_SMALL_INT_PROMOTION = + "hcat.data.tiny.small.int.promotion"; + public static final boolean HCAT_DATA_TINY_SMALL_INT_PROMOTION_DEFAULT = false; + + /** + * {@value} (default: {@value #HCAT_INPUT_BAD_RECORD_THRESHOLD_DEFAULT}). + * Threshold for the ratio of bad records that will be silently skipped without causing a task + * failure. This is useful when processing large data sets with corrupt records, when its + * acceptable to skip some bad records. + */ + public static final String HCAT_INPUT_BAD_RECORD_THRESHOLD_KEY = "hcat.input.bad.record.threshold"; + public static final float HCAT_INPUT_BAD_RECORD_THRESHOLD_DEFAULT = 0.0001f; + + /** + * {@value} (default: {@value #HCAT_INPUT_BAD_RECORD_MIN_DEFAULT}). + * Number of bad records that will be accepted before applying + * {@value #HCAT_INPUT_BAD_RECORD_THRESHOLD_KEY}. This is necessary to prevent an initial bad + * record from causing a task failure. + */ + public static final String HCAT_INPUT_BAD_RECORD_MIN_KEY = "hcat.input.bad.record.min"; + public static final int HCAT_INPUT_BAD_RECORD_MIN_DEFAULT = 2; +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/common/HCatContext.java hcatalog/core/src/main/java/org/apache/hcatalog/common/HCatContext.java new file mode 100644 index 0000000..b01301b --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/common/HCatContext.java @@ -0,0 +1,89 @@ +/** + * 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.hcatalog.common; + +import com.google.common.base.Optional; +import com.google.common.base.Preconditions; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.common.classification.InterfaceAudience; +import org.apache.hadoop.hive.common.classification.InterfaceStability; + +import java.util.Map; + +/** + * HCatContext is a singleton that provides global access to configuration data. + * + *

HCatalog provides a variety of functionality that users can configure at runtime through + * configuration properties. Available configuration properties are defined in + * {@link HCatConstants}. HCatContext allows users to enable optional functionality by + * setting properties in a provided configuration.

+ * + *

HCatalog users (MR apps, processing framework adapters) should set properties + * in a configuration that has been provided to + * {@link #setConf(org.apache.hadoop.conf.Configuration)} to enable optional functionality. + * The job configuration must be used to ensure properties are passed to the backend MR tasks.

+ * + *

HCatalog developers should enable optional functionality by checking properties + * from {@link #getConf()}. Since users are not obligated to set a configuration, optional + * functionality must provide a sensible default.

+ * + * @deprecated Use/modify {@link org.apache.hive.hcatalog.common.HCatContext} instead + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public enum HCatContext { + INSTANCE; + + private Configuration conf = null; + + /** + * Use the given configuration for optional behavior. Keys exclusive to an existing config + * are set in the new conf. The job conf must be used to ensure properties are passed to + * backend MR tasks. + */ + public synchronized HCatContext setConf(Configuration newConf) { + Preconditions.checkNotNull(newConf, "Required parameter 'newConf' must not be null."); + + if (conf == null) { + conf = newConf; + return this; + } + + if (conf != newConf) { + for (Map.Entry entry : conf) { + if ((entry.getKey().matches("hcat.*")) && (newConf.get(entry.getKey()) == null)) { + newConf.set(entry.getKey(), entry.getValue()); + } + } + conf = newConf; + } + return this; + } + + /** + * Get the configuration, if there is one. Users are not required to setup HCatContext + * unless they wish to override default behavior, so the configuration may not be present. + * + * @return an Optional that might contain a Configuration + */ + public Optional getConf() { + return Optional.fromNullable(conf); + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/common/HCatException.java hcatalog/core/src/main/java/org/apache/hcatalog/common/HCatException.java new file mode 100644 index 0000000..eeca28f --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/common/HCatException.java @@ -0,0 +1,160 @@ +/** + * 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.hcatalog.common; + +import java.io.IOException; + +/** + * Class representing exceptions thrown by HCat. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.common.HCatException} instead + */ +public class HCatException extends IOException { + + private static final long serialVersionUID = 1L; + + /** The error type enum for this exception. */ + private final ErrorType errorType; + + /** + * Instantiates a new hcat exception. + * @param errorType the error type + */ + public HCatException(ErrorType errorType) { + this(errorType, null, null); + } + + + /** + * Instantiates a new hcat exception. + * @param errorType the error type + * @param cause the cause + */ + public HCatException(ErrorType errorType, Throwable cause) { + this(errorType, null, cause); + } + + /** + * Instantiates a new hcat exception. + * @param errorType the error type + * @param extraMessage extra messages to add to the message string + */ + public HCatException(ErrorType errorType, String extraMessage) { + this(errorType, extraMessage, null); + } + + /** + * Instantiates a new hcat exception. + * @param errorType the error type + * @param extraMessage extra messages to add to the message string + * @param cause the cause + */ + public HCatException(ErrorType errorType, String extraMessage, Throwable cause) { + super(buildErrorMessage( + errorType, + extraMessage, + cause), cause); + this.errorType = errorType; + } + + + //TODO : remove default error type constructors after all exceptions + //are changed to use error types + + /** + * Instantiates a new hcat exception. + * @param message the error message + */ + public HCatException(String message) { + this(ErrorType.ERROR_INTERNAL_EXCEPTION, message, null); + } + + /** + * Instantiates a new hcat exception. + * @param message the error message + * @param cause the cause + */ + public HCatException(String message, Throwable cause) { + this(ErrorType.ERROR_INTERNAL_EXCEPTION, message, cause); + } + + + /** + * Builds the error message string. The error type message is appended with the extra message. If appendCause + * is true for the error type, then the message of the cause also is added to the message. + * @param type the error type + * @param extraMessage the extra message string + * @param cause the cause for the exception + * @return the exception message string + */ + public static String buildErrorMessage(ErrorType type, String extraMessage, Throwable cause) { + + //Initial message is just the error type message + StringBuffer message = new StringBuffer(HCatException.class.getName()); + message.append(" : " + type.getErrorCode()); + message.append(" : " + type.getErrorMessage()); + + if (extraMessage != null) { + //Add the extra message value to buffer + message.append(" : " + extraMessage); + } + + if (type.appendCauseMessage()) { + if (cause != null) { + //Add the cause message to buffer + message.append(". Cause : " + cause.toString()); + } + } + + return message.toString(); + } + + + /** + * Is this a retriable error. + * @return is it retriable + */ + public boolean isRetriable() { + return errorType.isRetriable(); + } + + /** + * Gets the error type. + * @return the error type enum + */ + public ErrorType getErrorType() { + return errorType; + } + + /** + * Gets the error code. + * @return the error code + */ + public int getErrorCode() { + return errorType.getErrorCode(); + } + + /* (non-Javadoc) + * @see java.lang.Throwable#toString() + */ + @Override + public String toString() { + return getMessage(); + } + +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/common/HCatUtil.java hcatalog/core/src/main/java/org/apache/hcatalog/common/HCatUtil.java new file mode 100644 index 0000000..850d7db --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/common/HCatUtil.java @@ -0,0 +1,630 @@ +/** + * 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.hcatalog.common; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.permission.FsAction; +import org.apache.hadoop.hive.common.JavaUtils; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.MetaStoreUtils; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat; +import org.apache.hadoop.hive.ql.metadata.Partition; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.plan.TableDesc; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; +import org.apache.hadoop.hive.thrift.DelegationTokenIdentifier; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapred.JobClient; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier; +import org.apache.hadoop.util.ReflectionUtils; +import org.apache.hcatalog.data.Pair; +import org.apache.hcatalog.data.schema.HCatFieldSchema; +import org.apache.hcatalog.data.schema.HCatSchema; +import org.apache.hcatalog.data.schema.HCatSchemaUtils; +import org.apache.hcatalog.mapreduce.FosterStorageHandler; +import org.apache.hcatalog.mapreduce.HCatOutputFormat; +import org.apache.hcatalog.mapreduce.HCatStorageHandler; +import org.apache.hcatalog.mapreduce.InputJobInfo; +import org.apache.hcatalog.mapreduce.OutputJobInfo; +import org.apache.hcatalog.mapreduce.PartInfo; +import org.apache.hcatalog.mapreduce.StorerInfo; +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.security.auth.login.LoginException; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.common.HCatUtil} instead + */ +public class HCatUtil { + + private static final Logger LOG = LoggerFactory.getLogger(HCatUtil.class); + private static volatile HiveClientCache hiveClientCache; + private final static int DEFAULT_HIVE_CACHE_EXPIRY_TIME_SECONDS = 2 * 60; + + public static boolean checkJobContextIfRunningFromBackend(JobContext j) { + if (j.getConfiguration().get("mapred.task.id", "").equals("") && + !("true".equals(j.getConfiguration().get("pig.illustrating")))) { + return false; + } + return true; + } + + public static String serialize(Serializable obj) throws IOException { + if (obj == null) { + return ""; + } + try { + ByteArrayOutputStream serialObj = new ByteArrayOutputStream(); + ObjectOutputStream objStream = new ObjectOutputStream(serialObj); + objStream.writeObject(obj); + objStream.close(); + return encodeBytes(serialObj.toByteArray()); + } catch (Exception e) { + throw new IOException("Serialization error: " + e.getMessage(), e); + } + } + + public static Object deserialize(String str) throws IOException { + if (str == null || str.length() == 0) { + return null; + } + try { + ByteArrayInputStream serialObj = new ByteArrayInputStream( + decodeBytes(str)); + ObjectInputStream objStream = new ObjectInputStream(serialObj); + return objStream.readObject(); + } catch (Exception e) { + throw new IOException("Deserialization error: " + e.getMessage(), e); + } + } + + public static String encodeBytes(byte[] bytes) { + StringBuffer strBuf = new StringBuffer(); + + for (int i = 0; i < bytes.length; i++) { + strBuf.append((char) (((bytes[i] >> 4) & 0xF) + ('a'))); + strBuf.append((char) (((bytes[i]) & 0xF) + ('a'))); + } + + return strBuf.toString(); + } + + public static byte[] decodeBytes(String str) { + byte[] bytes = new byte[str.length() / 2]; + for (int i = 0; i < str.length(); i += 2) { + char c = str.charAt(i); + bytes[i / 2] = (byte) ((c - 'a') << 4); + c = str.charAt(i + 1); + bytes[i / 2] += (c - 'a'); + } + return bytes; + } + + public static List getHCatFieldSchemaList( + FieldSchema... fields) throws HCatException { + List result = new ArrayList( + fields.length); + + for (FieldSchema f : fields) { + result.add(HCatSchemaUtils.getHCatFieldSchema(f)); + } + + return result; + } + + public static List getHCatFieldSchemaList( + List fields) throws HCatException { + if (fields == null) { + return null; + } else { + List result = new ArrayList(); + for (FieldSchema f : fields) { + result.add(HCatSchemaUtils.getHCatFieldSchema(f)); + } + return result; + } + } + + public static HCatSchema extractSchema(Table table) throws HCatException { + return new HCatSchema(HCatUtil.getHCatFieldSchemaList(table.getCols())); + } + + public static HCatSchema extractSchema(Partition partition) throws HCatException { + return new HCatSchema(HCatUtil.getHCatFieldSchemaList(partition.getCols())); + } + + public static List getFieldSchemaList( + List hcatFields) { + if (hcatFields == null) { + return null; + } else { + List result = new ArrayList(); + for (HCatFieldSchema f : hcatFields) { + result.add(HCatSchemaUtils.getFieldSchema(f)); + } + return result; + } + } + + public static Table getTable(HiveMetaStoreClient client, String dbName, String tableName) + throws NoSuchObjectException, TException, MetaException { + return new Table(client.getTable(dbName, tableName)); + } + + public static HCatSchema getTableSchemaWithPtnCols(Table table) throws IOException { + HCatSchema tableSchema = new HCatSchema(HCatUtil.getHCatFieldSchemaList(table.getCols())); + + if (table.getPartitionKeys().size() != 0) { + + // add partition keys to table schema + // NOTE : this assumes that we do not ever have ptn keys as columns + // inside the table schema as well! + for (FieldSchema fs : table.getPartitionKeys()) { + tableSchema.append(HCatSchemaUtils.getHCatFieldSchema(fs)); + } + } + return tableSchema; + } + + /** + * return the partition columns from a table instance + * + * @param table the instance to extract partition columns from + * @return HCatSchema instance which contains the partition columns + * @throws IOException + */ + public static HCatSchema getPartitionColumns(Table table) throws IOException { + HCatSchema cols = new HCatSchema(new LinkedList()); + if (table.getPartitionKeys().size() != 0) { + for (FieldSchema fs : table.getPartitionKeys()) { + cols.append(HCatSchemaUtils.getHCatFieldSchema(fs)); + } + } + return cols; + } + + /** + * Validate partition schema, checks if the column types match between the + * partition and the existing table schema. Returns the list of columns + * present in the partition but not in the table. + * + * @param table the table + * @param partitionSchema the partition schema + * @return the list of newly added fields + * @throws IOException Signals that an I/O exception has occurred. + */ + public static List validatePartitionSchema(Table table, + HCatSchema partitionSchema) throws IOException { + Map partitionKeyMap = new HashMap(); + + for (FieldSchema field : table.getPartitionKeys()) { + partitionKeyMap.put(field.getName().toLowerCase(), field); + } + + List tableCols = table.getCols(); + List newFields = new ArrayList(); + + for (int i = 0; i < partitionSchema.getFields().size(); i++) { + + FieldSchema field = HCatSchemaUtils.getFieldSchema(partitionSchema + .getFields().get(i)); + + FieldSchema tableField; + if (i < tableCols.size()) { + tableField = tableCols.get(i); + + if (!tableField.getName().equalsIgnoreCase(field.getName())) { + throw new HCatException( + ErrorType.ERROR_SCHEMA_COLUMN_MISMATCH, + "Expected column <" + tableField.getName() + + "> at position " + (i + 1) + + ", found column <" + field.getName() + + ">"); + } + } else { + tableField = partitionKeyMap.get(field.getName().toLowerCase()); + + if (tableField != null) { + throw new HCatException( + ErrorType.ERROR_SCHEMA_PARTITION_KEY, "Key <" + + field.getName() + ">"); + } + } + + if (tableField == null) { + // field present in partition but not in table + newFields.add(field); + } else { + // field present in both. validate type has not changed + TypeInfo partitionType = TypeInfoUtils + .getTypeInfoFromTypeString(field.getType()); + TypeInfo tableType = TypeInfoUtils + .getTypeInfoFromTypeString(tableField.getType()); + + if (!partitionType.equals(tableType)) { + throw new HCatException( + ErrorType.ERROR_SCHEMA_TYPE_MISMATCH, "Column <" + + field.getName() + ">, expected <" + + tableType.getTypeName() + ">, got <" + + partitionType.getTypeName() + ">"); + } + } + } + + return newFields; + } + + /** + * Test if the first FsAction is more permissive than the second. This is + * useful in cases where we want to ensure that a file owner has more + * permissions than the group they belong to, for eg. More completely(but + * potentially more cryptically) owner-r >= group-r >= world-r : bitwise + * and-masked with 0444 => 444 >= 440 >= 400 >= 000 owner-w >= group-w >= + * world-w : bitwise and-masked with &0222 => 222 >= 220 >= 200 >= 000 + * owner-x >= group-x >= world-x : bitwise and-masked with &0111 => 111 >= + * 110 >= 100 >= 000 + * + * @return true if first FsAction is more permissive than the second, false + * if not. + */ + public static boolean validateMorePermissive(FsAction first, FsAction second) { + if ((first == FsAction.ALL) || (second == FsAction.NONE) + || (first == second)) { + return true; + } + switch (first) { + case READ_EXECUTE: + return ((second == FsAction.READ) || (second == FsAction.EXECUTE)); + case READ_WRITE: + return ((second == FsAction.READ) || (second == FsAction.WRITE)); + case WRITE_EXECUTE: + return ((second == FsAction.WRITE) || (second == FsAction.EXECUTE)); + } + return false; + } + + /** + * Ensure that read or write permissions are not granted without also + * granting execute permissions. Essentially, r-- , rw- and -w- are invalid, + * r-x, -wx, rwx, ---, --x are valid + * + * @param perms The FsAction to verify + * @return true if the presence of read or write permission is accompanied + * by execute permissions + */ + public static boolean validateExecuteBitPresentIfReadOrWrite(FsAction perms) { + if ((perms == FsAction.READ) || (perms == FsAction.WRITE) + || (perms == FsAction.READ_WRITE)) { + return false; + } + return true; + } + + public static Token getJobTrackerDelegationToken( + Configuration conf, String userName) throws Exception { + // LOG.info("getJobTrackerDelegationToken("+conf+","+userName+")"); + JobClient jcl = new JobClient(new JobConf(conf, HCatOutputFormat.class)); + Token t = jcl + .getDelegationToken(new Text(userName)); + // LOG.info("got "+t); + return t; + + // return null; + } + + public static Token extractThriftToken( + String tokenStrForm, String tokenSignature) throws MetaException, + TException, IOException { + // LOG.info("extractThriftToken("+tokenStrForm+","+tokenSignature+")"); + Token t = new Token(); + t.decodeFromUrlString(tokenStrForm); + t.setService(new Text(tokenSignature)); + // LOG.info("returning "+t); + return t; + } + + /** + * Create an instance of a storage handler defined in storerInfo. If one cannot be found + * then FosterStorageHandler is used to encapsulate the InputFormat, OutputFormat and SerDe. + * This StorageHandler assumes the other supplied storage artifacts are for a file-based storage system. + * @param conf job's configuration will be used to configure the Configurable StorageHandler + * @param storerInfo StorerInfo to definining the StorageHandler and InputFormat, OutputFormat and SerDe + * @return storageHandler instance + * @throws IOException + */ + public static HCatStorageHandler getStorageHandler(Configuration conf, StorerInfo storerInfo) throws IOException { + return getStorageHandler(conf, + storerInfo.getStorageHandlerClass(), + storerInfo.getSerdeClass(), + storerInfo.getIfClass(), + storerInfo.getOfClass()); + } + + public static HCatStorageHandler getStorageHandler(Configuration conf, PartInfo partitionInfo) throws IOException { + return HCatUtil.getStorageHandler( + conf, + partitionInfo.getStorageHandlerClassName(), + partitionInfo.getSerdeClassName(), + partitionInfo.getInputFormatClassName(), + partitionInfo.getOutputFormatClassName()); + } + + /** + * Create an instance of a storage handler. If storageHandler == null, + * then surrrogate StorageHandler is used to encapsulate the InputFormat, OutputFormat and SerDe. + * This StorageHandler assumes the other supplied storage artifacts are for a file-based storage system. + * @param conf job's configuration will be used to configure the Configurable StorageHandler + * @param storageHandler fully qualified class name of the desired StorageHandle instance + * @param serDe fully qualified class name of the desired SerDe instance + * @param inputFormat fully qualified class name of the desired InputFormat instance + * @param outputFormat fully qualified class name of the desired outputFormat instance + * @return storageHandler instance + * @throws IOException + */ + public static HCatStorageHandler getStorageHandler(Configuration conf, + String storageHandler, + String serDe, + String inputFormat, + String outputFormat) + throws IOException { + + if ((storageHandler == null) || (storageHandler.equals(FosterStorageHandler.class.getName()))) { + try { + FosterStorageHandler fosterStorageHandler = + new FosterStorageHandler(inputFormat, outputFormat, serDe); + fosterStorageHandler.setConf(conf); + return fosterStorageHandler; + } catch (ClassNotFoundException e) { + throw new IOException("Failed to load " + + "foster storage handler", e); + } + } + + try { + Class handlerClass = + (Class) Class + .forName(storageHandler, true, JavaUtils.getClassLoader()); + return (HCatStorageHandler) ReflectionUtils.newInstance( + handlerClass, conf); + } catch (ClassNotFoundException e) { + throw new IOException("Error in loading storage handler." + + e.getMessage(), e); + } + } + + public static Pair getDbAndTableName(String tableName) throws IOException { + String[] dbTableNametokens = tableName.split("\\."); + if (dbTableNametokens.length == 1) { + return new Pair(MetaStoreUtils.DEFAULT_DATABASE_NAME, tableName); + } else if (dbTableNametokens.length == 2) { + return new Pair(dbTableNametokens[0], dbTableNametokens[1]); + } else { + throw new IOException("tableName expected in the form " + + ". or
. Got " + tableName); + } + } + + public static Map + getInputJobProperties(HCatStorageHandler storageHandler, + InputJobInfo inputJobInfo) { + TableDesc tableDesc = new TableDesc(storageHandler.getSerDeClass(), + storageHandler.getInputFormatClass(), + storageHandler.getOutputFormatClass(), + inputJobInfo.getTableInfo().getStorerInfo().getProperties()); + if (tableDesc.getJobProperties() == null) { + tableDesc.setJobProperties(new HashMap()); + } + + Map jobProperties = new HashMap(); + try { + tableDesc.getJobProperties().put( + HCatConstants.HCAT_KEY_JOB_INFO, + HCatUtil.serialize(inputJobInfo)); + + storageHandler.configureInputJobProperties(tableDesc, + jobProperties); + + } catch (IOException e) { + throw new IllegalStateException( + "Failed to configure StorageHandler", e); + } + + return jobProperties; + } + + @InterfaceAudience.Private + @InterfaceStability.Evolving + public static void + configureOutputStorageHandler(HCatStorageHandler storageHandler, + Configuration conf, + OutputJobInfo outputJobInfo) { + //TODO replace IgnoreKeyTextOutputFormat with a + //HiveOutputFormatWrapper in StorageHandler + TableDesc tableDesc = new TableDesc(storageHandler.getSerDeClass(), + storageHandler.getInputFormatClass(), + IgnoreKeyTextOutputFormat.class, + outputJobInfo.getTableInfo().getStorerInfo().getProperties()); + if (tableDesc.getJobProperties() == null) + tableDesc.setJobProperties(new HashMap()); + for (Map.Entry el : conf) { + tableDesc.getJobProperties().put(el.getKey(), el.getValue()); + } + + Map jobProperties = new HashMap(); + try { + tableDesc.getJobProperties().put( + HCatConstants.HCAT_KEY_OUTPUT_INFO, + HCatUtil.serialize(outputJobInfo)); + + storageHandler.configureOutputJobProperties(tableDesc, + jobProperties); + + for (Map.Entry el : jobProperties.entrySet()) { + conf.set(el.getKey(), el.getValue()); + } + } catch (IOException e) { + throw new IllegalStateException( + "Failed to configure StorageHandler", e); + } + } + + /** + * Replace the contents of dest with the contents of src + * @param src + * @param dest + */ + public static void copyConf(Configuration src, Configuration dest) { + dest.clear(); + for (Map.Entry el : src) { + dest.set(el.getKey(), el.getValue()); + } + } + + /** + * Get or create a hive client depending on whether it exits in cache or not + * @param hiveConf The hive configuration + * @return the client + * @throws MetaException When HiveMetaStoreClient couldn't be created + * @throws IOException + */ + public static HiveMetaStoreClient getHiveClient(HiveConf hiveConf) + throws MetaException, IOException { + + // Singleton behaviour: create the cache instance if required. The cache needs to be created lazily and + // using the expiry time available in hiveConf. + + if (hiveClientCache == null) { + synchronized (HiveMetaStoreClient.class) { + if (hiveClientCache == null) { + hiveClientCache = new HiveClientCache(hiveConf.getInt(HCatConstants.HCAT_HIVE_CLIENT_EXPIRY_TIME, + DEFAULT_HIVE_CACHE_EXPIRY_TIME_SECONDS)); + } + } + } + try { + return hiveClientCache.get(hiveConf); + } catch (LoginException e) { + throw new IOException("Couldn't create hiveMetaStoreClient, Error getting UGI for user", e); + } + } + + public static void closeHiveClientQuietly(HiveMetaStoreClient client) { + try { + if (client != null) + client.close(); + } catch (Exception e) { + LOG.debug("Error closing metastore client. Ignored the error.", e); + } + } + + public static HiveConf getHiveConf(Configuration conf) + throws IOException { + + HiveConf hiveConf = new HiveConf(conf, HCatUtil.class); + + //copy the hive conf into the job conf and restore it + //in the backend context + if (conf.get(HCatConstants.HCAT_KEY_HIVE_CONF) == null) { + conf.set(HCatConstants.HCAT_KEY_HIVE_CONF, + HCatUtil.serialize(hiveConf.getAllProperties())); + } else { + //Copy configuration properties into the hive conf + Properties properties = (Properties) HCatUtil.deserialize( + conf.get(HCatConstants.HCAT_KEY_HIVE_CONF)); + + for (Map.Entry prop : properties.entrySet()) { + if (prop.getValue() instanceof String) { + hiveConf.set((String) prop.getKey(), (String) prop.getValue()); + } else if (prop.getValue() instanceof Integer) { + hiveConf.setInt((String) prop.getKey(), + (Integer) prop.getValue()); + } else if (prop.getValue() instanceof Boolean) { + hiveConf.setBoolean((String) prop.getKey(), + (Boolean) prop.getValue()); + } else if (prop.getValue() instanceof Long) { + hiveConf.setLong((String) prop.getKey(), (Long) prop.getValue()); + } else if (prop.getValue() instanceof Float) { + hiveConf.setFloat((String) prop.getKey(), + (Float) prop.getValue()); + } + } + } + + if (conf.get(HCatConstants.HCAT_KEY_TOKEN_SIGNATURE) != null) { + hiveConf.set("hive.metastore.token.signature", + conf.get(HCatConstants.HCAT_KEY_TOKEN_SIGNATURE)); + } + + return hiveConf; + } + + + public static JobConf getJobConfFromContext(JobContext jobContext) { + JobConf jobConf; + // we need to convert the jobContext into a jobConf + // 0.18 jobConf (Hive) vs 0.20+ jobContext (HCat) + // begin conversion.. + jobConf = new JobConf(jobContext.getConfiguration()); + // ..end of conversion + + + return jobConf; + } + + public static void copyJobPropertiesToJobConf( + Map jobProperties, JobConf jobConf) { + for (Map.Entry entry : jobProperties.entrySet()) { + jobConf.set(entry.getKey(), entry.getValue()); + } + } + + + public static boolean isHadoop23() { + String version = org.apache.hadoop.util.VersionInfo.getVersion(); + if (version.matches("\\b0\\.23\\..+\\b")||version.matches("\\b2\\..*")) + return true; + return false; + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/common/HiveClientCache.java hcatalog/core/src/main/java/org/apache/hcatalog/common/HiveClientCache.java new file mode 100644 index 0000000..96fe8f1 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/common/HiveClientCache.java @@ -0,0 +1,338 @@ +/** + * 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.hcatalog.common; + +import com.google.common.cache.Cache; +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.RemovalListener; +import com.google.common.cache.RemovalNotification; +import org.apache.commons.lang.builder.EqualsBuilder; +import org.apache.commons.lang.builder.HashCodeBuilder; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.security.auth.login.LoginException; +import java.io.IOException; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * A thread safe time expired cache for HiveMetaStoreClient + * @deprecated Use/modify {@link org.apache.hive.hcatalog.common.HiveClientCache} instead + */ +class HiveClientCache { + final private Cache hiveCache; + private static final Logger LOG = LoggerFactory.getLogger(HiveClientCache.class); + private final int timeout; + // This lock is used to make sure removalListener won't close a client that is being contemplated for returning by get() + private final Object CACHE_TEARDOWN_LOCK = new Object(); + + private static final AtomicInteger nextId = new AtomicInteger(0); + + // Since HiveMetaStoreClient is not threadsafe, hive clients are not shared across threads. + // Thread local variable containing each thread's unique ID, is used as one of the keys for the cache + // causing each thread to get a different client even if the hiveConf is same. + private static final ThreadLocal threadId = + new ThreadLocal() { + @Override + protected Integer initialValue() { + return nextId.getAndIncrement(); + } + }; + + private int getThreadId() { + return threadId.get(); + } + + /** + * @param timeout the length of time in seconds after a client is created that it should be automatically removed + */ + public HiveClientCache(final int timeout) { + this.timeout = timeout; + RemovalListener removalListener = + new RemovalListener() { + public void onRemoval(RemovalNotification notification) { + CacheableHiveMetaStoreClient hiveMetaStoreClient = notification.getValue(); + if (hiveMetaStoreClient != null) { + synchronized (CACHE_TEARDOWN_LOCK) { + hiveMetaStoreClient.setExpiredFromCache(); + hiveMetaStoreClient.tearDownIfUnused(); + } + } + } + }; + hiveCache = CacheBuilder.newBuilder() + .expireAfterWrite(timeout, TimeUnit.SECONDS) + .removalListener(removalListener) + .build(); + + // Add a shutdown hook for cleanup, if there are elements remaining in the cache which were not cleaned up. + // This is the best effort approach. Ignore any error while doing so. Notice that most of the clients + // would get cleaned up via either the removalListener or the close() call, only the active clients + // that are in the cache or expired but being used in other threads wont get cleaned. The following code will only + // clean the active cache ones. The ones expired from cache but being hold by other threads are in the mercy + // of finalize() being called. + Thread cleanupHiveClientShutdownThread = new Thread() { + @Override + public void run() { + LOG.debug("Cleaning up hive client cache in ShutDown hook"); + closeAllClientsQuietly(); + } + }; + Runtime.getRuntime().addShutdownHook(cleanupHiveClientShutdownThread); + } + + /** + * Note: This doesn't check if they are being used or not, meant only to be called during shutdown etc. + */ + void closeAllClientsQuietly() { + try { + ConcurrentMap elements = hiveCache.asMap(); + for (CacheableHiveMetaStoreClient cacheableHiveMetaStoreClient : elements.values()) { + cacheableHiveMetaStoreClient.tearDown(); + } + } catch (Exception e) { + LOG.warn("Clean up of hive clients in the cache failed. Ignored", e); + } + } + + public void cleanup() { + hiveCache.cleanUp(); + } + + /** + * Returns a cached client if exists or else creates one, caches and returns it. It also checks that the client is + * healthy and can be reused + * @param hiveConf + * @return the hive client + * @throws MetaException + * @throws IOException + * @throws LoginException + */ + public HiveMetaStoreClient get(final HiveConf hiveConf) throws MetaException, IOException, LoginException { + final HiveClientCacheKey cacheKey = HiveClientCacheKey.fromHiveConf(hiveConf, getThreadId()); + CacheableHiveMetaStoreClient hiveMetaStoreClient = null; + // the hmsc is not shared across threads. So the only way it could get closed while we are doing healthcheck + // is if removalListener closes it. The synchronization takes care that removalListener won't do it + synchronized (CACHE_TEARDOWN_LOCK) { + hiveMetaStoreClient = getOrCreate(cacheKey); + hiveMetaStoreClient.acquire(); + } + if (!hiveMetaStoreClient.isOpen()) { + synchronized (CACHE_TEARDOWN_LOCK) { + hiveCache.invalidate(cacheKey); + hiveMetaStoreClient.close(); + hiveMetaStoreClient = getOrCreate(cacheKey); + hiveMetaStoreClient.acquire(); + } + } + return hiveMetaStoreClient; + } + + /** + * Return from cache if exists else create/cache and return + * @param cacheKey + * @return + * @throws IOException + * @throws MetaException + * @throws LoginException + */ + private CacheableHiveMetaStoreClient getOrCreate(final HiveClientCacheKey cacheKey) throws IOException, MetaException, LoginException { + try { + return hiveCache.get(cacheKey, new Callable() { + @Override + public CacheableHiveMetaStoreClient call() throws MetaException { + return new CacheableHiveMetaStoreClient(cacheKey.getHiveConf(), timeout); + } + }); + } catch (ExecutionException e) { + Throwable t = e.getCause(); + if (t instanceof IOException) { + throw (IOException) t; + } else if (t instanceof MetaException) { + throw (MetaException) t; + } else if (t instanceof LoginException) { + throw (LoginException) t; + } else { + throw new IOException("Error creating hiveMetaStoreClient", t); + } + } + } + + /** + * A class to wrap HiveConf and expose equality based only on UserGroupInformation and the metaStoreURIs. + * This becomes the key for the cache and this way the same HiveMetaStoreClient would be returned if + * UserGroupInformation and metaStoreURIs are same. This function can evolve to express + * the cases when HiveConf is different but the same hiveMetaStoreClient can be used + */ + public static class HiveClientCacheKey { + final private String metaStoreURIs; + final private UserGroupInformation ugi; + final private HiveConf hiveConf; + final private int threadId; + + private HiveClientCacheKey(HiveConf hiveConf, final int threadId) throws IOException, LoginException { + this.metaStoreURIs = hiveConf.getVar(HiveConf.ConfVars.METASTOREURIS); + ugi = ShimLoader.getHadoopShims().getUGIForConf(hiveConf); + this.hiveConf = hiveConf; + this.threadId = threadId; + } + + public static HiveClientCacheKey fromHiveConf(HiveConf hiveConf, final int threadId) throws IOException, LoginException { + return new HiveClientCacheKey(hiveConf, threadId); + } + + public HiveConf getHiveConf() { + return hiveConf; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + HiveClientCacheKey that = (HiveClientCacheKey) o; + return new EqualsBuilder(). + append(this.metaStoreURIs, + that.metaStoreURIs). + append(this.ugi, that.ugi). + append(this.threadId, that.threadId).isEquals(); + } + + @Override + public int hashCode() { + return new HashCodeBuilder(). + append(metaStoreURIs). + append(ugi). + append(threadId).toHashCode(); + } + } + + /** + * Add # of current users on HiveMetaStoreClient, so that the client can be cleaned when no one is using it. + */ + public static class CacheableHiveMetaStoreClient extends HiveMetaStoreClient { + private AtomicInteger users = new AtomicInteger(0); + private volatile boolean expiredFromCache = false; + private boolean isClosed = false; + private final long expiryTime; + private static final int EXPIRY_TIME_EXTENSION_IN_MILLIS = 60 * 1000; + + public CacheableHiveMetaStoreClient(final HiveConf conf, final int timeout) throws MetaException { + super(conf); + // Extend the expiry time with some extra time on top of guava expiry time to make sure + // that items closed() are for sure expired and would never be returned by guava. + this.expiryTime = System.currentTimeMillis() + timeout * 1000 + EXPIRY_TIME_EXTENSION_IN_MILLIS; + } + + private void acquire() { + users.incrementAndGet(); + } + + private void release() { + users.decrementAndGet(); + } + + public void setExpiredFromCache() { + expiredFromCache = true; + } + + public boolean isClosed() { + return isClosed; + } + + /** + * Make a call to hive meta store and see if the client is still usable. Some calls where the user provides + * invalid data renders the client unusable for future use (example: create a table with very long table name) + * @return + */ + protected boolean isOpen() { + try { + // Look for an unlikely database name and see if either MetaException or TException is thrown + this.getDatabase("NonExistentDatabaseUsedForHealthCheck"); + } catch (NoSuchObjectException e) { + return true; // It is okay if the database doesn't exist + } catch (MetaException e) { + return false; + } catch (TException e) { + return false; + } + return true; + } + + /** + * Decrement the user count and piggyback this to set expiry flag as well, then teardown(), if conditions are met. + * This *MUST* be called by anyone who uses this client. + */ + @Override + public void close() { + release(); + if (System.currentTimeMillis() >= expiryTime) + setExpiredFromCache(); + tearDownIfUnused(); + } + + /** + * Tear down only if + * 1. There are no active user + * 2. It has expired from the cache + */ + private void tearDownIfUnused() { + if (users.get() == 0 && expiredFromCache) { + this.tearDown(); + } + } + + /** + * Close if not closed already + */ + protected synchronized void tearDown() { + try { + if (!isClosed) { + super.close(); + } + isClosed = true; + } catch (Exception e) { + LOG.warn("Error closing hive metastore client. Ignored.", e); + } + } + + /** + * Last effort to clean up, may not even get called. + * @throws Throwable + */ + @Override + protected void finalize() throws Throwable { + try { + this.tearDown(); + } finally { + super.finalize(); + } + } + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/data/DataType.java hcatalog/core/src/main/java/org/apache/hcatalog/data/DataType.java new file mode 100644 index 0000000..6b435fb --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/data/DataType.java @@ -0,0 +1,209 @@ +/** + * 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.hcatalog.data; + +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import java.util.Map.Entry; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.data.DataType} instead + */ +public abstract class DataType { + + public static final byte NULL = 1; + public static final byte BOOLEAN = 5; + public static final byte BYTE = 6; + public static final byte INTEGER = 10; + public static final byte SHORT = 11; + public static final byte LONG = 15; + public static final byte FLOAT = 20; + public static final byte DOUBLE = 25; + public static final byte STRING = 55; + public static final byte BINARY = 60; + + public static final byte MAP = 100; + public static final byte STRUCT = 110; + public static final byte LIST = 120; + public static final byte ERROR = -1; + + /** + * Determine the datatype of an object. + * @param o Object to test. + * @return byte code of the type, or ERROR if we don't know. + */ + public static byte findType(Object o) { + if (o == null) { + return NULL; + } + + Class clazz = o.getClass(); + + // Try to put the most common first + if (clazz == String.class) { + return STRING; + } else if (clazz == Integer.class) { + return INTEGER; + } else if (clazz == Long.class) { + return LONG; + } else if (clazz == Float.class) { + return FLOAT; + } else if (clazz == Double.class) { + return DOUBLE; + } else if (clazz == Boolean.class) { + return BOOLEAN; + } else if (clazz == Byte.class) { + return BYTE; + } else if (clazz == Short.class) { + return SHORT; + } else if (o instanceof List) { + return LIST; + } else if (o instanceof Map) { + return MAP; + } else if (o instanceof byte[]) { + return BINARY; + } else { + return ERROR; + } + } + + public static int compare(Object o1, Object o2) { + + return compare(o1, o2, findType(o1), findType(o2)); + } + + public static int compare(Object o1, Object o2, byte dt1, byte dt2) { + if (dt1 == dt2) { + switch (dt1) { + case NULL: + return 0; + + case BOOLEAN: + return ((Boolean) o1).compareTo((Boolean) o2); + + case BYTE: + return ((Byte) o1).compareTo((Byte) o2); + + case INTEGER: + return ((Integer) o1).compareTo((Integer) o2); + + case LONG: + return ((Long) o1).compareTo((Long) o2); + + case FLOAT: + return ((Float) o1).compareTo((Float) o2); + + case DOUBLE: + return ((Double) o1).compareTo((Double) o2); + + case STRING: + return ((String) o1).compareTo((String) o2); + + case SHORT: + return ((Short) o1).compareTo((Short) o2); + + case BINARY: + return compareByteArray((byte[]) o1, (byte[]) o2); + + case LIST: + List l1 = (List) o1; + List l2 = (List) o2; + int len = l1.size(); + if (len != l2.size()) { + return len - l2.size(); + } else { + for (int i = 0; i < len; i++) { + int cmpVal = compare(l1.get(i), l2.get(i)); + if (cmpVal != 0) { + return cmpVal; + } + } + return 0; + } + + case MAP: { + Map m1 = (Map) o1; + Map m2 = (Map) o2; + int sz1 = m1.size(); + int sz2 = m2.size(); + if (sz1 < sz2) { + return -1; + } else if (sz1 > sz2) { + return 1; + } else { + // This is bad, but we have to sort the keys of the maps in order + // to be commutative. + TreeMap tm1 = new TreeMap(m1); + TreeMap tm2 = new TreeMap(m2); + Iterator> i1 = tm1.entrySet().iterator(); + Iterator> i2 = tm2.entrySet().iterator(); + while (i1.hasNext()) { + Map.Entry entry1 = i1.next(); + Map.Entry entry2 = i2.next(); + int c = compare(entry1.getValue(), entry2.getValue()); + if (c != 0) { + return c; + } else { + c = compare(entry1.getValue(), entry2.getValue()); + if (c != 0) { + return c; + } + } + } + return 0; + } + } + + default: + throw new RuntimeException("Unkown type " + dt1 + + " in compare"); + } + } else { + return dt1 < dt2 ? -1 : 1; + } + } + + private static int compareByteArray(byte[] o1, byte[] o2) { + + for (int i = 0; i < o1.length; i++) { + if (i == o2.length) { + return 1; + } + if (o1[i] == o2[i]) { + continue; + } + if (o1[i] > o1[i]) { + return 1; + } else { + return -1; + } + } + + //bytes in o1 are same as o2 + //in case o2 was longer + if (o2.length > o1.length) { + return -1; + } + return 0; //equals + } + +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/data/DefaultHCatRecord.java hcatalog/core/src/main/java/org/apache/hcatalog/data/DefaultHCatRecord.java new file mode 100644 index 0000000..dc211b1 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/data/DefaultHCatRecord.java @@ -0,0 +1,133 @@ +/** + * 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.hcatalog.data; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.data.schema.HCatSchema; +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.data.DefaultHCatRecord} instead + */ +public class DefaultHCatRecord extends HCatRecord { + + private List contents; + + public DefaultHCatRecord() { + contents = new ArrayList(); + } + + public DefaultHCatRecord(int size) { + contents = new ArrayList(size); + for (int i = 0; i < size; i++) { + contents.add(null); + } + } + + @Override + public void remove(int idx) throws HCatException { + contents.remove(idx); + } + + public DefaultHCatRecord(List list) { + contents = list; + } + + @Override + public Object get(int fieldNum) { + return contents.get(fieldNum); + } + + @Override + public List getAll() { + return contents; + } + + @Override + public void set(int fieldNum, Object val) { + contents.set(fieldNum, val); + } + + @Override + public int size() { + return contents.size(); + } + + @Override + public void readFields(DataInput in) throws IOException { + + contents.clear(); + int len = in.readInt(); + for (int i = 0; i < len; i++) { + contents.add(ReaderWriter.readDatum(in)); + } + } + + @Override + public void write(DataOutput out) throws IOException { + int sz = size(); + out.writeInt(sz); + for (int i = 0; i < sz; i++) { + ReaderWriter.writeDatum(out, contents.get(i)); + } + + } + + @Override + public int hashCode() { + int hash = 1; + for (Object o : contents) { + if (o != null) { + hash = 31 * hash + o.hashCode(); + } + } + return hash; + } + + @Override + public String toString() { + + StringBuilder sb = new StringBuilder(); + for (Object o : contents) { + sb.append(o + "\t"); + } + return sb.toString(); + } + + @Override + public Object get(String fieldName, HCatSchema recordSchema) throws HCatException { + return get(recordSchema.getPosition(fieldName)); + } + + @Override + public void set(String fieldName, HCatSchema recordSchema, Object value) throws HCatException { + set(recordSchema.getPosition(fieldName), value); + } + + @Override + public void copy(HCatRecord r) throws HCatException { + this.contents = r.getAll(); + } + +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/data/HCatRecord.java hcatalog/core/src/main/java/org/apache/hcatalog/data/HCatRecord.java new file mode 100644 index 0000000..5cf57e0 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/data/HCatRecord.java @@ -0,0 +1,151 @@ +/** + * 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.hcatalog.data; + +import java.util.List; +import java.util.Map; + +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.data.schema.HCatSchema; + +/** + * Abstract class exposing get and set semantics for basic record usage. + * Note : + * HCatRecord is designed only to be used as in-memory representation only. + * Don't use it to store data on the physical device. + + * @deprecated Use/modify {@link org.apache.hive.hcatalog.data.HCatRecord} instead + */ +public abstract class HCatRecord implements HCatRecordable { + + public abstract Object get(String fieldName, HCatSchema recordSchema) throws HCatException; + + public abstract void set(String fieldName, HCatSchema recordSchema, Object value) throws HCatException; + + public abstract void remove(int idx) throws HCatException; + + public abstract void copy(HCatRecord r) throws HCatException; + + protected Object get(String fieldName, HCatSchema recordSchema, Class clazz) throws HCatException { + // TODO : if needed, verify that recordschema entry for fieldname matches appropriate type. + return get(fieldName, recordSchema); + } + + public Boolean getBoolean(String fieldName, HCatSchema recordSchema) throws HCatException { + return (Boolean) get(fieldName, recordSchema, Boolean.class); + } + + public void setBoolean(String fieldName, HCatSchema recordSchema, Boolean value) throws HCatException { + set(fieldName, recordSchema, value); + } + + public byte[] getByteArray(String fieldName, HCatSchema recordSchema) throws HCatException { + return (byte[]) get(fieldName, recordSchema, byte[].class); + } + + public void setByteArray(String fieldName, HCatSchema recordSchema, byte[] value) throws HCatException { + set(fieldName, recordSchema, value); + } + + public Byte getByte(String fieldName, HCatSchema recordSchema) throws HCatException { + //TINYINT + return (Byte) get(fieldName, recordSchema, Byte.class); + } + + public void setByte(String fieldName, HCatSchema recordSchema, Byte value) throws HCatException { + set(fieldName, recordSchema, value); + } + + public Short getShort(String fieldName, HCatSchema recordSchema) throws HCatException { + // SMALLINT + return (Short) get(fieldName, recordSchema, Short.class); + } + + public void setShort(String fieldName, HCatSchema recordSchema, Short value) throws HCatException { + set(fieldName, recordSchema, value); + } + + public Integer getInteger(String fieldName, HCatSchema recordSchema) throws HCatException { + return (Integer) get(fieldName, recordSchema, Integer.class); + } + + public void setInteger(String fieldName, HCatSchema recordSchema, Integer value) throws HCatException { + set(fieldName, recordSchema, value); + } + + public Long getLong(String fieldName, HCatSchema recordSchema) throws HCatException { + // BIGINT + return (Long) get(fieldName, recordSchema, Long.class); + } + + public void setLong(String fieldName, HCatSchema recordSchema, Long value) throws HCatException { + set(fieldName, recordSchema, value); + } + + public Float getFloat(String fieldName, HCatSchema recordSchema) throws HCatException { + return (Float) get(fieldName, recordSchema, Float.class); + } + + public void setFloat(String fieldName, HCatSchema recordSchema, Float value) throws HCatException { + set(fieldName, recordSchema, value); + } + + public Double getDouble(String fieldName, HCatSchema recordSchema) throws HCatException { + return (Double) get(fieldName, recordSchema, Double.class); + } + + public void setDouble(String fieldName, HCatSchema recordSchema, Double value) throws HCatException { + set(fieldName, recordSchema, value); + } + + public String getString(String fieldName, HCatSchema recordSchema) throws HCatException { + return (String) get(fieldName, recordSchema, String.class); + } + + public void setString(String fieldName, HCatSchema recordSchema, String value) throws HCatException { + set(fieldName, recordSchema, value); + } + + @SuppressWarnings("unchecked") + public List getStruct(String fieldName, HCatSchema recordSchema) throws HCatException { + return (List) get(fieldName, recordSchema, List.class); + } + + public void setStruct(String fieldName, HCatSchema recordSchema, List value) throws HCatException { + set(fieldName, recordSchema, value); + } + + public List getList(String fieldName, HCatSchema recordSchema) throws HCatException { + return (List) get(fieldName, recordSchema, List.class); + } + + public void setList(String fieldName, HCatSchema recordSchema, List value) throws HCatException { + set(fieldName, recordSchema, value); + } + + public Map getMap(String fieldName, HCatSchema recordSchema) throws HCatException { + return (Map) get(fieldName, recordSchema, Map.class); + } + + public void setMap(String fieldName, HCatSchema recordSchema, Map value) throws HCatException { + set(fieldName, recordSchema, value); + } + +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/data/HCatRecordObjectInspector.java hcatalog/core/src/main/java/org/apache/hcatalog/data/HCatRecordObjectInspector.java new file mode 100644 index 0000000..3d138d4 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/data/HCatRecordObjectInspector.java @@ -0,0 +1,56 @@ +/** + * 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.hcatalog.data; + +import java.util.List; + +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.StandardStructObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.StructField; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.data.HCatRecordObjectInspector} instead + */ +public class HCatRecordObjectInspector extends StandardStructObjectInspector { + + protected HCatRecordObjectInspector(List structFieldNames, + List structFieldObjectInspectors) { + super(structFieldNames, structFieldObjectInspectors); + } + + @Override + public Object getStructFieldData(Object data, StructField fieldRef) { + if (data == null) { + return new IllegalArgumentException("Data passed in to get field from was null!"); + } + + int fieldID = ((MyField) fieldRef).getFieldID(); + if (!(fieldID >= 0 && fieldID < fields.size())) { + throw new IllegalArgumentException("Invalid field index [" + fieldID + "]"); + } + + return ((HCatRecord) data).get(fieldID); + } + + @Override + public List getStructFieldsDataAsList(Object o) { + return ((HCatRecord) o).getAll(); + } + +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/data/HCatRecordObjectInspectorFactory.java hcatalog/core/src/main/java/org/apache/hcatalog/data/HCatRecordObjectInspectorFactory.java new file mode 100644 index 0000000..da4b0f8 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/data/HCatRecordObjectInspectorFactory.java @@ -0,0 +1,133 @@ +/** + * 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.hcatalog.data; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; + +import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * ObjectInspectorFactory for HCatRecordObjectInspectors (and associated helper inspectors) + * @deprecated Use/modify {@link org.apache.hive.hcatalog.data.HCatRecordObjectInspectorFactory} instead + */ +public class HCatRecordObjectInspectorFactory { + + private final static Logger LOG = LoggerFactory.getLogger(HCatRecordObjectInspectorFactory.class); + + static HashMap cachedHCatRecordObjectInspectors = + new HashMap(); + static HashMap cachedObjectInspectors = + new HashMap(); + + /** + * Returns HCatRecordObjectInspector given a StructTypeInfo type definition for the record to look into + * @param typeInfo Type definition for the record to look into + * @return appropriate HCatRecordObjectInspector + * @throws SerDeException + */ + public static HCatRecordObjectInspector getHCatRecordObjectInspector( + StructTypeInfo typeInfo) throws SerDeException { + HCatRecordObjectInspector oi = cachedHCatRecordObjectInspectors.get(typeInfo); + if (oi == null) { + + LOG.debug("Got asked for OI for {} [{} ]", typeInfo.getCategory(), typeInfo.getTypeName()); + switch (typeInfo.getCategory()) { + case STRUCT: + StructTypeInfo structTypeInfo = (StructTypeInfo) typeInfo; + List fieldNames = structTypeInfo.getAllStructFieldNames(); + List fieldTypeInfos = structTypeInfo.getAllStructFieldTypeInfos(); + List fieldObjectInspectors = new ArrayList(fieldTypeInfos.size()); + for (int i = 0; i < fieldTypeInfos.size(); i++) { + fieldObjectInspectors.add(getStandardObjectInspectorFromTypeInfo(fieldTypeInfos.get(i))); + } + oi = new HCatRecordObjectInspector(fieldNames, fieldObjectInspectors); + + break; + default: + // Hmm.. not good, + // the only type expected here is STRUCT, which maps to HCatRecord + // - anything else is an error. Return null as the inspector. + throw new SerDeException("TypeInfo [" + typeInfo.getTypeName() + + "] was not of struct type - HCatRecord expected struct type, got [" + + typeInfo.getCategory().toString() + "]"); + } + cachedHCatRecordObjectInspectors.put(typeInfo, oi); + } + return oi; + } + + public static ObjectInspector getStandardObjectInspectorFromTypeInfo(TypeInfo typeInfo) { + + + ObjectInspector oi = cachedObjectInspectors.get(typeInfo); + if (oi == null) { + + LOG.debug("Got asked for OI for {}, [{}]", typeInfo.getCategory(), typeInfo.getTypeName()); + switch (typeInfo.getCategory()) { + case PRIMITIVE: + oi = PrimitiveObjectInspectorFactory.getPrimitiveJavaObjectInspector( + ((PrimitiveTypeInfo) typeInfo).getPrimitiveCategory()); + break; + case STRUCT: + StructTypeInfo structTypeInfo = (StructTypeInfo) typeInfo; + List fieldNames = structTypeInfo.getAllStructFieldNames(); + List fieldTypeInfos = structTypeInfo.getAllStructFieldTypeInfos(); + List fieldObjectInspectors = + new ArrayList(fieldTypeInfos.size()); + for (int i = 0; i < fieldTypeInfos.size(); i++) { + fieldObjectInspectors.add(getStandardObjectInspectorFromTypeInfo(fieldTypeInfos.get(i))); + } + oi = ObjectInspectorFactory.getStandardStructObjectInspector( + fieldNames, fieldObjectInspectors + ); + break; + case LIST: + ObjectInspector elementObjectInspector = getStandardObjectInspectorFromTypeInfo( + ((ListTypeInfo) typeInfo).getListElementTypeInfo()); + oi = ObjectInspectorFactory.getStandardListObjectInspector(elementObjectInspector); + break; + case MAP: + ObjectInspector keyObjectInspector = getStandardObjectInspectorFromTypeInfo( + ((MapTypeInfo) typeInfo).getMapKeyTypeInfo()); + ObjectInspector valueObjectInspector = getStandardObjectInspectorFromTypeInfo( + ((MapTypeInfo) typeInfo).getMapValueTypeInfo()); + oi = ObjectInspectorFactory.getStandardMapObjectInspector(keyObjectInspector, valueObjectInspector); + break; + default: + oi = null; + } + cachedObjectInspectors.put(typeInfo, oi); + } + return oi; + } + + +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/data/HCatRecordSerDe.java hcatalog/core/src/main/java/org/apache/hcatalog/data/HCatRecordSerDe.java new file mode 100644 index 0000000..f86da7f --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/data/HCatRecordSerDe.java @@ -0,0 +1,318 @@ +/** + * 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.hcatalog.data; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.TreeMap; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.serde.serdeConstants; +import org.apache.hadoop.hive.serde2.SerDe; +import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.SerDeStats; +import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.MapObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category; +import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.StructField; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; +import org.apache.hadoop.io.Writable; +import org.apache.hcatalog.common.HCatConstants; +import org.apache.hcatalog.common.HCatContext; +import org.apache.hcatalog.data.schema.HCatSchema; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * SerDe class for serializing to and from HCatRecord + * @deprecated Use/modify {@link org.apache.hive.hcatalog.data.HCatRecordSerDe} instead + */ +public class HCatRecordSerDe implements SerDe { + + private static final Logger LOG = LoggerFactory.getLogger(HCatRecordSerDe.class); + + public HCatRecordSerDe() throws SerDeException { + } + + private List columnNames; + private List columnTypes; + private StructTypeInfo rowTypeInfo; + + private HCatRecordObjectInspector cachedObjectInspector; + + @Override + public void initialize(Configuration conf, Properties tbl) + throws SerDeException { + + LOG.debug("Initializing HCatRecordSerDe"); + LOG.debug("props to serde: {}", tbl.entrySet()); + + // Get column names and types + String columnNameProperty = tbl.getProperty(serdeConstants.LIST_COLUMNS); + String columnTypeProperty = tbl.getProperty(serdeConstants.LIST_COLUMN_TYPES); + + // all table column names + if (columnNameProperty.length() == 0) { + columnNames = new ArrayList(); + } else { + columnNames = Arrays.asList(columnNameProperty.split(",")); + } + + // all column types + if (columnTypeProperty.length() == 0) { + columnTypes = new ArrayList(); + } else { + columnTypes = TypeInfoUtils.getTypeInfosFromTypeString(columnTypeProperty); + } + + + LOG.debug("columns: {} {}", columnNameProperty, columnNames); + LOG.debug("types: {} {}", columnTypeProperty, columnTypes); + assert (columnNames.size() == columnTypes.size()); + + rowTypeInfo = (StructTypeInfo) TypeInfoFactory.getStructTypeInfo(columnNames, columnTypes); + cachedObjectInspector = HCatRecordObjectInspectorFactory.getHCatRecordObjectInspector(rowTypeInfo); + } + + public void initialize(HCatSchema hsch) throws SerDeException { + + LOG.debug("Initializing HCatRecordSerDe through HCatSchema {}.", hsch); + + rowTypeInfo = (StructTypeInfo) TypeInfoUtils.getTypeInfoFromTypeString(hsch.getSchemaAsTypeString()); + cachedObjectInspector = HCatRecordObjectInspectorFactory.getHCatRecordObjectInspector(rowTypeInfo); + + } + + + /** + * The purpose of a deserialize method is to turn a data blob + * which is a writable representation of the data into an + * object that can then be parsed using the appropriate + * ObjectInspector. In this case, since HCatRecord is directly + * already the Writable object, there's no extra work to be done + * here. Most of the logic resides in the ObjectInspector to be + * able to return values from within the HCatRecord to hive when + * it wants it. + */ + @Override + public Object deserialize(Writable data) throws SerDeException { + if (!(data instanceof HCatRecord)) { + throw new SerDeException(getClass().getName() + ": expects HCatRecord!"); + } + + return (HCatRecord) data; + } + + /** + * The purpose of the serialize method is to turn an object-representation + * with a provided ObjectInspector into a Writable format, which + * the underlying layer can then use to write out. + * + * In this case, it means that Hive will call this method to convert + * an object with appropriate objectinspectors that it knows about, + * to write out a HCatRecord. + */ + @Override + public Writable serialize(Object obj, ObjectInspector objInspector) + throws SerDeException { + if (objInspector.getCategory() != Category.STRUCT) { + throw new SerDeException(getClass().toString() + + " can only serialize struct types, but we got: " + + objInspector.getTypeName()); + } + return new DefaultHCatRecord((List) serializeStruct(obj, (StructObjectInspector) objInspector)); + } + + + /** + * Return serialized HCatRecord from an underlying + * object-representation, and readable by an ObjectInspector + * @param obj : Underlying object-representation + * @param soi : StructObjectInspector + * @return HCatRecord + */ + private static List serializeStruct(Object obj, StructObjectInspector soi) + throws SerDeException { + + List fields = soi.getAllStructFieldRefs(); + List list = soi.getStructFieldsDataAsList(obj); + + if (list == null) { + return null; + } + + List l = new ArrayList(fields.size()); + + if (fields != null) { + for (int i = 0; i < fields.size(); i++) { + + // Get the field objectInspector and the field object. + ObjectInspector foi = fields.get(i).getFieldObjectInspector(); + Object f = list.get(i); + Object res = serializeField(f, foi); + l.add(i, res); + } + } + return l; + } + + /** + * Return underlying Java Object from an object-representation + * that is readable by a provided ObjectInspector. + */ + public static Object serializeField(Object field, ObjectInspector fieldObjectInspector) + throws SerDeException { + + Object res; + if (fieldObjectInspector.getCategory() == Category.PRIMITIVE) { + res = serializePrimitiveField(field, fieldObjectInspector); + } else if (fieldObjectInspector.getCategory() == Category.STRUCT) { + res = serializeStruct(field, (StructObjectInspector) fieldObjectInspector); + } else if (fieldObjectInspector.getCategory() == Category.LIST) { + res = serializeList(field, (ListObjectInspector) fieldObjectInspector); + } else if (fieldObjectInspector.getCategory() == Category.MAP) { + res = serializeMap(field, (MapObjectInspector) fieldObjectInspector); + } else { + throw new SerDeException(HCatRecordSerDe.class.toString() + + " does not know what to do with fields of unknown category: " + + fieldObjectInspector.getCategory() + " , type: " + fieldObjectInspector.getTypeName()); + } + return res; + } + + /** + * Helper method to return underlying Java Map from + * an object-representation that is readable by a provided + * MapObjectInspector + */ + private static Map serializeMap(Object f, MapObjectInspector moi) throws SerDeException { + ObjectInspector koi = moi.getMapKeyObjectInspector(); + ObjectInspector voi = moi.getMapValueObjectInspector(); + Map m = new TreeMap(); + + Map readMap = moi.getMap(f); + if (readMap == null) { + return null; + } else { + for (Map.Entry entry : readMap.entrySet()) { + m.put(serializeField(entry.getKey(), koi), serializeField(entry.getValue(), voi)); + } + } + return m; + } + + private static List serializeList(Object f, ListObjectInspector loi) throws SerDeException { + List l = loi.getList(f); + if (l == null) { + return null; + } + + ObjectInspector eloi = loi.getListElementObjectInspector(); + if (eloi.getCategory() == Category.PRIMITIVE) { + List list = new ArrayList(l.size()); + for (int i = 0; i < l.size(); i++) { + list.add(((PrimitiveObjectInspector) eloi).getPrimitiveJavaObject(l.get(i))); + } + return list; + } else if (eloi.getCategory() == Category.STRUCT) { + List> list = new ArrayList>(l.size()); + for (int i = 0; i < l.size(); i++) { + list.add(serializeStruct(l.get(i), (StructObjectInspector) eloi)); + } + return list; + } else if (eloi.getCategory() == Category.LIST) { + List> list = new ArrayList>(l.size()); + for (int i = 0; i < l.size(); i++) { + list.add(serializeList(l.get(i), (ListObjectInspector) eloi)); + } + return list; + } else if (eloi.getCategory() == Category.MAP) { + List> list = new ArrayList>(l.size()); + for (int i = 0; i < l.size(); i++) { + list.add(serializeMap(l.get(i), (MapObjectInspector) eloi)); + } + return list; + } else { + throw new SerDeException(HCatRecordSerDe.class.toString() + + " does not know what to do with fields of unknown category: " + + eloi.getCategory() + " , type: " + eloi.getTypeName()); + } + } + + private static Object serializePrimitiveField(Object field, + ObjectInspector fieldObjectInspector) { + + if (field != null && HCatContext.INSTANCE.getConf().isPresent()) { + Configuration conf = HCatContext.INSTANCE.getConf().get(); + + if (field instanceof Boolean && + conf.getBoolean( + HCatConstants.HCAT_DATA_CONVERT_BOOLEAN_TO_INTEGER, + HCatConstants.HCAT_DATA_CONVERT_BOOLEAN_TO_INTEGER_DEFAULT)) { + return ((Boolean) field) ? 1 : 0; + } else if (field instanceof Short && + conf.getBoolean( + HCatConstants.HCAT_DATA_TINY_SMALL_INT_PROMOTION, + HCatConstants.HCAT_DATA_TINY_SMALL_INT_PROMOTION_DEFAULT)) { + return new Integer((Short) field); + } else if (field instanceof Byte && + conf.getBoolean( + HCatConstants.HCAT_DATA_TINY_SMALL_INT_PROMOTION, + HCatConstants.HCAT_DATA_TINY_SMALL_INT_PROMOTION_DEFAULT)) { + return new Integer((Byte) field); + } + } + + return ((PrimitiveObjectInspector) fieldObjectInspector).getPrimitiveJavaObject(field); + } + + /** + * Return an object inspector that can read through the object + * that we return from deserialize(). To wit, that means we need + * to return an ObjectInspector that can read HCatRecord, given + * the type info for it during initialize(). This also means + * that this method cannot and should not be called before initialize() + */ + @Override + public ObjectInspector getObjectInspector() throws SerDeException { + return (ObjectInspector) cachedObjectInspector; + } + + @Override + public Class getSerializedClass() { + return HCatRecord.class; + } + + @Override + public SerDeStats getSerDeStats() { + // no support for statistics yet + return null; + } + + +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/data/HCatRecordable.java hcatalog/core/src/main/java/org/apache/hcatalog/data/HCatRecordable.java new file mode 100644 index 0000000..e9f0a25 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/data/HCatRecordable.java @@ -0,0 +1,57 @@ +/** + * 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.hcatalog.data; + +import java.util.List; + +import org.apache.hadoop.io.Writable; + +/** + * Interface that determines whether we can implement a HCatRecord on top of it + * @deprecated Use/modify {@link org.apache.hive.hcatalog.data.HCatRecordable} instead + */ +public interface HCatRecordable extends Writable { + + /** + * Gets the field at the specified index. + * @param fieldNum the field number + * @return the object at the specified index + */ + Object get(int fieldNum); + + /** + * Gets all the fields of the hcat record. + * @return the list of fields + */ + List getAll(); + + /** + * Sets the field at the specified index. + * @param fieldNum the field number + * @param value the value to set + */ + void set(int fieldNum, Object value); + + /** + * Gets the size of the hcat record. + * @return the size + */ + int size(); + +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/data/JsonSerDe.java hcatalog/core/src/main/java/org/apache/hcatalog/data/JsonSerDe.java new file mode 100644 index 0000000..66bde04 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/data/JsonSerDe.java @@ -0,0 +1,578 @@ +/** + * 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.hcatalog.data; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.serde.serdeConstants; +import org.apache.hadoop.hive.serde2.SerDe; +import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.SerDeStats; +import org.apache.hadoop.hive.serde2.SerDeUtils; +import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.MapObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.StructField; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.UnionObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.BooleanObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.ByteObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.DoubleObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.FloatObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.IntObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.ShortObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.StringObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampObjectInspector; +import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.Writable; +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.data.schema.HCatFieldSchema; +import org.apache.hcatalog.data.schema.HCatFieldSchema.Type; +import org.apache.hcatalog.data.schema.HCatSchema; +import org.apache.hcatalog.data.schema.HCatSchemaUtils; +import org.codehaus.jackson.JsonFactory; +import org.codehaus.jackson.JsonParseException; +import org.codehaus.jackson.JsonParser; +import org.codehaus.jackson.JsonToken; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.data.JsonSerDe} instead + */ +public class JsonSerDe implements SerDe { + + private static final Logger LOG = LoggerFactory.getLogger(JsonSerDe.class); + private List columnNames; + private List columnTypes; + + private StructTypeInfo rowTypeInfo; + private HCatSchema schema; + + private JsonFactory jsonFactory = null; + + private HCatRecordObjectInspector cachedObjectInspector; + + @Override + public void initialize(Configuration conf, Properties tbl) + throws SerDeException { + + + LOG.debug("Initializing JsonSerDe"); + LOG.debug("props to serde: {}", tbl.entrySet()); + + + // Get column names and types + String columnNameProperty = tbl.getProperty(serdeConstants.LIST_COLUMNS); + String columnTypeProperty = tbl.getProperty(serdeConstants.LIST_COLUMN_TYPES); + + // all table column names + if (columnNameProperty.length() == 0) { + columnNames = new ArrayList(); + } else { + columnNames = Arrays.asList(columnNameProperty.split(",")); + } + + // all column types + if (columnTypeProperty.length() == 0) { + columnTypes = new ArrayList(); + } else { + columnTypes = TypeInfoUtils.getTypeInfosFromTypeString(columnTypeProperty); + } + + LOG.debug("columns: {}, {}", columnNameProperty, columnNames); + LOG.debug("types: {}, {} ", columnTypeProperty, columnTypes); + + assert (columnNames.size() == columnTypes.size()); + + rowTypeInfo = (StructTypeInfo) TypeInfoFactory.getStructTypeInfo(columnNames, columnTypes); + + cachedObjectInspector = HCatRecordObjectInspectorFactory.getHCatRecordObjectInspector(rowTypeInfo); + try { + schema = HCatSchemaUtils.getHCatSchema(rowTypeInfo).get(0).getStructSubSchema(); + LOG.debug("schema : {}", schema); + LOG.debug("fields : {}", schema.getFieldNames()); + } catch (HCatException e) { + throw new SerDeException(e); + } + + jsonFactory = new JsonFactory(); + } + + /** + * Takes JSON string in Text form, and has to return an object representation above + * it that's readable by the corresponding object inspector. + * + * For this implementation, since we're using the jackson parser, we can construct + * our own object implementation, and we use HCatRecord for it + */ + @Override + public Object deserialize(Writable blob) throws SerDeException { + + Text t = (Text) blob; + JsonParser p; + List r = new ArrayList(Collections.nCopies(columnNames.size(), null)); + try { + p = jsonFactory.createJsonParser(new ByteArrayInputStream((t.getBytes()))); + if (p.nextToken() != JsonToken.START_OBJECT) { + throw new IOException("Start token not found where expected"); + } + JsonToken token; + while (((token = p.nextToken()) != JsonToken.END_OBJECT) && (token != null)) { + // iterate through each token, and create appropriate object here. + populateRecord(r, token, p, schema); + } + } catch (JsonParseException e) { + LOG.warn("Error [{}] parsing json text [{}].", e, t); + LOG.debug(null, e); + throw new SerDeException(e); + } catch (IOException e) { + LOG.warn("Error [{}] parsing json text [{}].", e, t); + LOG.debug(null, e); + throw new SerDeException(e); + } + + return new DefaultHCatRecord(r); + } + + private void populateRecord(List r, JsonToken token, JsonParser p, HCatSchema s) throws IOException { + if (token != JsonToken.FIELD_NAME) { + throw new IOException("Field name expected"); + } + String fieldName = p.getText(); + int fpos; + try { + fpos = s.getPosition(fieldName); + } catch (NullPointerException npe) { + fpos = getPositionFromHiveInternalColumnName(fieldName); + LOG.debug("NPE finding position for field [{}] in schema [{}]", fieldName, s); + if (!fieldName.equalsIgnoreCase(getHiveInternalColumnName(fpos))) { + LOG.error("Hive internal column name {} and position " + + "encoding {} for the column name are at odds", fieldName, fpos); + throw npe; + } + if (fpos == -1) { + return; // unknown field, we return. + } + } + HCatFieldSchema hcatFieldSchema = s.getFields().get(fpos); + Object currField = extractCurrentField(p, null, hcatFieldSchema, false); + r.set(fpos, currField); + } + + public String getHiveInternalColumnName(int fpos) { + return HiveConf.getColumnInternalName(fpos); + } + + public int getPositionFromHiveInternalColumnName(String internalName) { +// return HiveConf.getPositionFromInternalName(fieldName); + // The above line should have been all the implementation that + // we need, but due to a bug in that impl which recognizes + // only single-digit columns, we need another impl here. + Pattern internalPattern = Pattern.compile("_col([0-9]+)"); + Matcher m = internalPattern.matcher(internalName); + if (!m.matches()) { + return -1; + } else { + return Integer.parseInt(m.group(1)); + } + } + + /** + * Utility method to extract current expected field from given JsonParser + * + * To get the field, we need either a type or a hcatFieldSchema(necessary for complex types) + * It is possible that one of them can be null, and so, if so, the other is instantiated + * from the other + * + * isTokenCurrent is a boolean variable also passed in, which determines + * if the JsonParser is already at the token we expect to read next, or + * needs advancing to the next before we read. + */ + private Object extractCurrentField(JsonParser p, Type t, + HCatFieldSchema hcatFieldSchema, boolean isTokenCurrent) throws IOException, JsonParseException, + HCatException { + Object val = null; + JsonToken valueToken; + if (isTokenCurrent) { + valueToken = p.getCurrentToken(); + } else { + valueToken = p.nextToken(); + } + + if (hcatFieldSchema != null) { + t = hcatFieldSchema.getType(); + } + switch (t) { + case INT: + val = (valueToken == JsonToken.VALUE_NULL) ? null : p.getIntValue(); + break; + case TINYINT: + val = (valueToken == JsonToken.VALUE_NULL) ? null : p.getByteValue(); + break; + case SMALLINT: + val = (valueToken == JsonToken.VALUE_NULL) ? null : p.getShortValue(); + break; + case BIGINT: + val = (valueToken == JsonToken.VALUE_NULL) ? null : p.getLongValue(); + break; + case BOOLEAN: + String bval = (valueToken == JsonToken.VALUE_NULL) ? null : p.getText(); + if (bval != null) { + val = Boolean.valueOf(bval); + } else { + val = null; + } + break; + case FLOAT: + val = (valueToken == JsonToken.VALUE_NULL) ? null : p.getFloatValue(); + break; + case DOUBLE: + val = (valueToken == JsonToken.VALUE_NULL) ? null : p.getDoubleValue(); + break; + case STRING: + val = (valueToken == JsonToken.VALUE_NULL) ? null : p.getText(); + break; + case BINARY: + throw new IOException("JsonSerDe does not support BINARY type"); + case ARRAY: + if (valueToken == JsonToken.VALUE_NULL) { + val = null; + break; + } + if (valueToken != JsonToken.START_ARRAY) { + throw new IOException("Start of Array expected"); + } + List arr = new ArrayList(); + while ((valueToken = p.nextToken()) != JsonToken.END_ARRAY) { + arr.add(extractCurrentField(p, null, hcatFieldSchema.getArrayElementSchema().get(0), true)); + } + val = arr; + break; + case MAP: + if (valueToken == JsonToken.VALUE_NULL) { + val = null; + break; + } + if (valueToken != JsonToken.START_OBJECT) { + throw new IOException("Start of Object expected"); + } + Map map = new LinkedHashMap(); + Type keyType = hcatFieldSchema.getMapKeyType(); + HCatFieldSchema valueSchema = hcatFieldSchema.getMapValueSchema().get(0); + while ((valueToken = p.nextToken()) != JsonToken.END_OBJECT) { + Object k = getObjectOfCorrespondingPrimitiveType(p.getCurrentName(), keyType); + Object v; + if (valueSchema.getType() == HCatFieldSchema.Type.STRUCT) { + v = extractCurrentField(p, null, valueSchema, false); + } else { + v = extractCurrentField(p, null, valueSchema, true); + } + + map.put(k, v); + } + val = map; + break; + case STRUCT: + if (valueToken == JsonToken.VALUE_NULL) { + val = null; + break; + } + if (valueToken != JsonToken.START_OBJECT) { + throw new IOException("Start of Object expected"); + } + HCatSchema subSchema = hcatFieldSchema.getStructSubSchema(); + int sz = subSchema.getFieldNames().size(); + + List struct = new ArrayList(Collections.nCopies(sz, null)); + while ((valueToken = p.nextToken()) != JsonToken.END_OBJECT) { + populateRecord(struct, valueToken, p, subSchema); + } + val = struct; + break; + } + return val; + } + + private Object getObjectOfCorrespondingPrimitiveType(String s, Type t) throws IOException { + switch (t) { + case INT: + return Integer.valueOf(s); + case TINYINT: + return Byte.valueOf(s); + case SMALLINT: + return Short.valueOf(s); + case BIGINT: + return Long.valueOf(s); + case BOOLEAN: + return (s.equalsIgnoreCase("true")); + case FLOAT: + return Float.valueOf(s); + case DOUBLE: + return Double.valueOf(s); + case STRING: + return s; + case BINARY: + throw new IOException("JsonSerDe does not support BINARY type"); + } + throw new IOException("Could not convert from string to map type " + t); + } + + /** + * Given an object and object inspector pair, traverse the object + * and generate a Text representation of the object. + */ + @Override + public Writable serialize(Object obj, ObjectInspector objInspector) + throws SerDeException { + StringBuilder sb = new StringBuilder(); + try { + + StructObjectInspector soi = (StructObjectInspector) objInspector; + List structFields = soi.getAllStructFieldRefs(); + assert (columnNames.size() == structFields.size()); + if (obj == null) { + sb.append("null"); + } else { + sb.append(SerDeUtils.LBRACE); + for (int i = 0; i < structFields.size(); i++) { + if (i > 0) { + sb.append(SerDeUtils.COMMA); + } + sb.append(SerDeUtils.QUOTE); + sb.append(columnNames.get(i)); + sb.append(SerDeUtils.QUOTE); + sb.append(SerDeUtils.COLON); + buildJSONString(sb, soi.getStructFieldData(obj, structFields.get(i)), + structFields.get(i).getFieldObjectInspector()); + } + sb.append(SerDeUtils.RBRACE); + } + + } catch (IOException e) { + LOG.warn("Error generating json text from object.", e); + throw new SerDeException(e); + } + return new Text(sb.toString()); + } + + // TODO : code section copied over from SerDeUtils because of non-standard json production there + // should use quotes for all field names. We should fix this there, and then remove this copy. + // See http://jackson.codehaus.org/1.7.3/javadoc/org/codehaus/jackson/JsonParser.Feature.html#ALLOW_UNQUOTED_FIELD_NAMES + // for details - trying to enable Jackson to ignore that doesn't seem to work(compilation failure + // when attempting to use that feature, so having to change the production itself. + // Also, throws IOException when Binary is detected. + private static void buildJSONString(StringBuilder sb, Object o, ObjectInspector oi) throws IOException { + + switch (oi.getCategory()) { + case PRIMITIVE: { + PrimitiveObjectInspector poi = (PrimitiveObjectInspector) oi; + if (o == null) { + sb.append("null"); + } else { + switch (poi.getPrimitiveCategory()) { + case BOOLEAN: { + boolean b = ((BooleanObjectInspector) poi).get(o); + sb.append(b ? "true" : "false"); + break; + } + case BYTE: { + sb.append(((ByteObjectInspector) poi).get(o)); + break; + } + case SHORT: { + sb.append(((ShortObjectInspector) poi).get(o)); + break; + } + case INT: { + sb.append(((IntObjectInspector) poi).get(o)); + break; + } + case LONG: { + sb.append(((LongObjectInspector) poi).get(o)); + break; + } + case FLOAT: { + sb.append(((FloatObjectInspector) poi).get(o)); + break; + } + case DOUBLE: { + sb.append(((DoubleObjectInspector) poi).get(o)); + break; + } + case STRING: { + sb.append('"'); + sb.append(SerDeUtils.escapeString(((StringObjectInspector) poi) + .getPrimitiveJavaObject(o))); + sb.append('"'); + break; + } + case TIMESTAMP: { + sb.append('"'); + sb.append(((TimestampObjectInspector) poi) + .getPrimitiveWritableObject(o)); + sb.append('"'); + break; + } + case BINARY: { + throw new IOException("JsonSerDe does not support BINARY type"); + } + default: + throw new RuntimeException("Unknown primitive type: " + + poi.getPrimitiveCategory()); + } + } + break; + } + case LIST: { + ListObjectInspector loi = (ListObjectInspector) oi; + ObjectInspector listElementObjectInspector = loi + .getListElementObjectInspector(); + List olist = loi.getList(o); + if (olist == null) { + sb.append("null"); + } else { + sb.append(SerDeUtils.LBRACKET); + for (int i = 0; i < olist.size(); i++) { + if (i > 0) { + sb.append(SerDeUtils.COMMA); + } + buildJSONString(sb, olist.get(i), listElementObjectInspector); + } + sb.append(SerDeUtils.RBRACKET); + } + break; + } + case MAP: { + MapObjectInspector moi = (MapObjectInspector) oi; + ObjectInspector mapKeyObjectInspector = moi.getMapKeyObjectInspector(); + ObjectInspector mapValueObjectInspector = moi + .getMapValueObjectInspector(); + Map omap = moi.getMap(o); + if (omap == null) { + sb.append("null"); + } else { + sb.append(SerDeUtils.LBRACE); + boolean first = true; + for (Object entry : omap.entrySet()) { + if (first) { + first = false; + } else { + sb.append(SerDeUtils.COMMA); + } + Map.Entry e = (Map.Entry) entry; + StringBuilder keyBuilder = new StringBuilder(); + buildJSONString(keyBuilder, e.getKey(), mapKeyObjectInspector); + String keyString = keyBuilder.toString().trim(); + boolean doQuoting = (!keyString.isEmpty()) && (keyString.charAt(0) != SerDeUtils.QUOTE); + if (doQuoting) { + sb.append(SerDeUtils.QUOTE); + } + sb.append(keyString); + if (doQuoting) { + sb.append(SerDeUtils.QUOTE); + } + sb.append(SerDeUtils.COLON); + buildJSONString(sb, e.getValue(), mapValueObjectInspector); + } + sb.append(SerDeUtils.RBRACE); + } + break; + } + case STRUCT: { + StructObjectInspector soi = (StructObjectInspector) oi; + List structFields = soi.getAllStructFieldRefs(); + if (o == null) { + sb.append("null"); + } else { + sb.append(SerDeUtils.LBRACE); + for (int i = 0; i < structFields.size(); i++) { + if (i > 0) { + sb.append(SerDeUtils.COMMA); + } + sb.append(SerDeUtils.QUOTE); + sb.append(structFields.get(i).getFieldName()); + sb.append(SerDeUtils.QUOTE); + sb.append(SerDeUtils.COLON); + buildJSONString(sb, soi.getStructFieldData(o, structFields.get(i)), + structFields.get(i).getFieldObjectInspector()); + } + sb.append(SerDeUtils.RBRACE); + } + break; + } + case UNION: { + UnionObjectInspector uoi = (UnionObjectInspector) oi; + if (o == null) { + sb.append("null"); + } else { + sb.append(SerDeUtils.LBRACE); + sb.append(uoi.getTag(o)); + sb.append(SerDeUtils.COLON); + buildJSONString(sb, uoi.getField(o), + uoi.getObjectInspectors().get(uoi.getTag(o))); + sb.append(SerDeUtils.RBRACE); + } + break; + } + default: + throw new RuntimeException("Unknown type in ObjectInspector!"); + } + } + + + /** + * Returns an object inspector for the specified schema that + * is capable of reading in the object representation of the JSON string + */ + @Override + public ObjectInspector getObjectInspector() throws SerDeException { + return cachedObjectInspector; + } + + @Override + public Class getSerializedClass() { + return Text.class; + } + + @Override + public SerDeStats getSerDeStats() { + // no support for statistics yet + return null; + } + +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/data/LazyHCatRecord.java hcatalog/core/src/main/java/org/apache/hcatalog/data/LazyHCatRecord.java new file mode 100644 index 0000000..421c149 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/data/LazyHCatRecord.java @@ -0,0 +1,147 @@ +/** + * 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.hcatalog.data; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.StructField; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.data.schema.HCatSchema; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An implementation of HCatRecord that wraps an Object returned by a SerDe + * and an ObjectInspector. This delays deserialization of unused columns. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.data.LazyHCatRecord} instead + */ +public class LazyHCatRecord extends HCatRecord { + + public static final Logger LOG = LoggerFactory.getLogger(LazyHCatRecord.class.getName()); + + private Object wrappedObject; + private StructObjectInspector soi; + + @Override + public Object get(int fieldNum) { + try { + StructField fref = soi.getAllStructFieldRefs().get(fieldNum); + return HCatRecordSerDe.serializeField( + soi.getStructFieldData(wrappedObject, fref), + fref.getFieldObjectInspector()); + } catch (SerDeException e) { + throw new IllegalStateException("SerDe Exception deserializing",e); + } + } + + @Override + public List getAll() { + List r = new ArrayList(this.size()); + for (int i = 0; i < this.size(); i++){ + r.add(i, get(i)); + } + return r; + } + + @Override + public void set(int fieldNum, Object value) { + throw new UnsupportedOperationException("not allowed to run set() on LazyHCatRecord"); + } + + @Override + public int size() { + return soi.getAllStructFieldRefs().size(); + } + + @Override + public void readFields(DataInput in) throws IOException { + throw new UnsupportedOperationException("LazyHCatRecord is intended to wrap" + + " an object/object inspector as a HCatRecord " + + "- it does not need to be read from DataInput."); + } + + @Override + public void write(DataOutput out) throws IOException { + throw new UnsupportedOperationException("LazyHCatRecord is intended to wrap" + + " an object/object inspector as a HCatRecord " + + "- it does not need to be written to a DataOutput."); + } + + @Override + public Object get(String fieldName, HCatSchema recordSchema) throws HCatException { + int idx = recordSchema.getPosition(fieldName); + return get(idx); + } + + @Override + public void set(String fieldName, HCatSchema recordSchema, Object value) throws HCatException { + throw new UnsupportedOperationException("not allowed to run set() on LazyHCatRecord"); + } + + @Override + public void remove(int idx) throws HCatException { + throw new UnsupportedOperationException("not allowed to run remove() on LazyHCatRecord"); + } + + @Override + public void copy(HCatRecord r) throws HCatException { + throw new UnsupportedOperationException("not allowed to run copy() on LazyHCatRecord"); + } + + public LazyHCatRecord(Object wrappedObject, ObjectInspector oi) throws Exception { + if (oi.getCategory() != Category.STRUCT) { + throw new SerDeException(getClass().toString() + + " can only make a lazy hcat record from " + + "objects of struct types, but we got: " + oi.getTypeName()); + } + + this.soi = (StructObjectInspector)oi; + this.wrappedObject = wrappedObject; + } + + @Override + public String toString(){ + StringBuilder sb = new StringBuilder(); + for(int i = 0; i< size() ; i++) { + sb.append(get(i)+"\t"); + } + return sb.toString(); + } + + /** + * Convert this LazyHCatRecord to a DefaultHCatRecord. This is required + * before you can write out a record via write. + * @return an HCatRecord that can be serialized + * @throws HCatException + */ + public HCatRecord getWritable() throws HCatException { + DefaultHCatRecord d = new DefaultHCatRecord(); + d.copy(this); + return d; + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/data/Pair.java hcatalog/core/src/main/java/org/apache/hcatalog/data/Pair.java new file mode 100644 index 0000000..ba553c3 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/data/Pair.java @@ -0,0 +1,90 @@ +/** + * 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.hcatalog.data; + +import java.io.Serializable; + +/** + * Copy of C++ STL pair container. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.data.Pair} instead + */ +public class Pair implements Serializable { + + private static final long serialVersionUID = 1L; + public T first; + public U second; + + /** + * @param f First element in pair. + * @param s Second element in pair. + */ + public Pair(T f, U s) { + first = f; + second = s; + } + + /* (non-Javadoc) + * @see java.lang.Object#toString() + */ + @Override + public String toString() { + return "[" + first.toString() + "," + second.toString() + "]"; + } + + @Override + public int hashCode() { + return (((this.first == null ? 1 : this.first.hashCode()) * 17) + + (this.second == null ? 1 : this.second.hashCode()) * 19); + } + + @Override + public boolean equals(Object other) { + if (other == null) { + return false; + } + + if (!(other instanceof Pair)) { + return false; + } + + Pair otherPair = (Pair) other; + + if (this.first == null) { + if (otherPair.first != null) { + return false; + } else { + return true; + } + } + + if (this.second == null) { + if (otherPair.second != null) { + return false; + } else { + return true; + } + } + + if (this.first.equals(otherPair.first) && this.second.equals(otherPair.second)) { + return true; + } else { + return false; + } + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/data/ReaderWriter.java hcatalog/core/src/main/java/org/apache/hcatalog/data/ReaderWriter.java new file mode 100644 index 0000000..5aba66f --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/data/ReaderWriter.java @@ -0,0 +1,195 @@ +/** + * 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.hcatalog.data; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.hadoop.io.VIntWritable; +import org.apache.hadoop.io.VLongWritable; + + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.data.ReaderWriter} instead + */ +public abstract class ReaderWriter { + + private static final String UTF8 = "UTF-8"; + + public static Object readDatum(DataInput in) throws IOException { + + byte type = in.readByte(); + switch (type) { + + case DataType.STRING: + byte[] buffer = new byte[in.readInt()]; + in.readFully(buffer); + return new String(buffer, UTF8); + + case DataType.INTEGER: + VIntWritable vint = new VIntWritable(); + vint.readFields(in); + return vint.get(); + + case DataType.LONG: + VLongWritable vlong = new VLongWritable(); + vlong.readFields(in); + return vlong.get(); + + case DataType.FLOAT: + return in.readFloat(); + + case DataType.DOUBLE: + return in.readDouble(); + + case DataType.BOOLEAN: + return in.readBoolean(); + + case DataType.BYTE: + return in.readByte(); + + case DataType.SHORT: + return in.readShort(); + + case DataType.NULL: + return null; + + case DataType.BINARY: + int len = in.readInt(); + byte[] ba = new byte[len]; + in.readFully(ba); + return ba; + + case DataType.MAP: + int size = in.readInt(); + Map m = new HashMap(size); + for (int i = 0; i < size; i++) { + m.put(readDatum(in), readDatum(in)); + } + return m; + + case DataType.LIST: + int sz = in.readInt(); + List list = new ArrayList(sz); + for (int i = 0; i < sz; i++) { + list.add(readDatum(in)); + } + return list; + + default: + throw new IOException("Unexpected data type " + type + + " found in stream."); + } + } + + public static void writeDatum(DataOutput out, Object val) throws IOException { + // write the data type + byte type = DataType.findType(val); + switch (type) { + case DataType.LIST: + out.writeByte(DataType.LIST); + List list = (List) val; + int sz = list.size(); + out.writeInt(sz); + for (int i = 0; i < sz; i++) { + writeDatum(out, list.get(i)); + } + return; + + case DataType.MAP: + out.writeByte(DataType.MAP); + Map m = (Map) val; + out.writeInt(m.size()); + Iterator i = + m.entrySet().iterator(); + while (i.hasNext()) { + Entry entry = (Entry) i.next(); + writeDatum(out, entry.getKey()); + writeDatum(out, entry.getValue()); + } + return; + + case DataType.INTEGER: + out.writeByte(DataType.INTEGER); + new VIntWritable((Integer) val).write(out); + return; + + case DataType.LONG: + out.writeByte(DataType.LONG); + new VLongWritable((Long) val).write(out); + return; + + case DataType.FLOAT: + out.writeByte(DataType.FLOAT); + out.writeFloat((Float) val); + return; + + case DataType.DOUBLE: + out.writeByte(DataType.DOUBLE); + out.writeDouble((Double) val); + return; + + case DataType.BOOLEAN: + out.writeByte(DataType.BOOLEAN); + out.writeBoolean((Boolean) val); + return; + + case DataType.BYTE: + out.writeByte(DataType.BYTE); + out.writeByte((Byte) val); + return; + + case DataType.SHORT: + out.writeByte(DataType.SHORT); + out.writeShort((Short) val); + return; + + case DataType.STRING: + String s = (String) val; + byte[] utfBytes = s.getBytes(ReaderWriter.UTF8); + out.writeByte(DataType.STRING); + out.writeInt(utfBytes.length); + out.write(utfBytes); + return; + + case DataType.BINARY: + byte[] ba = (byte[]) val; + out.writeByte(DataType.BINARY); + out.writeInt(ba.length); + out.write(ba); + return; + + case DataType.NULL: + out.writeByte(DataType.NULL); + return; + + default: + throw new IOException("Unexpected data type " + type + + " found in stream."); + } + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/data/schema/HCatFieldSchema.java hcatalog/core/src/main/java/org/apache/hcatalog/data/schema/HCatFieldSchema.java new file mode 100644 index 0000000..94f1317 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/data/schema/HCatFieldSchema.java @@ -0,0 +1,285 @@ +/** + * 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.hcatalog.data.schema; + +import java.io.Serializable; + +import org.apache.commons.lang.builder.ToStringBuilder; +import org.apache.hcatalog.common.HCatException; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.data.schema.HCatFieldSchema} instead + */ +public class HCatFieldSchema implements Serializable { + + public enum Type { + INT, + TINYINT, + SMALLINT, + BIGINT, + BOOLEAN, + FLOAT, + DOUBLE, + STRING, + ARRAY, + MAP, + STRUCT, + BINARY, + } + + public enum Category { + PRIMITIVE, + ARRAY, + MAP, + STRUCT; + + public static Category fromType(Type type) { + if (Type.ARRAY == type) { + return ARRAY; + } else if (Type.STRUCT == type) { + return STRUCT; + } else if (Type.MAP == type) { + return MAP; + } else { + return PRIMITIVE; + } + } + } + + ; + + public boolean isComplex() { + return (category == Category.PRIMITIVE) ? false : true; + } + + /** + * + */ + private static final long serialVersionUID = 1L; + + String fieldName = null; + String comment = null; + Type type = null; + Category category = null; + + // Populated if column is struct, array or map types. + // If struct type, contains schema of the struct. + // If array type, contains schema of one of the elements. + // If map type, contains schema of the value element. + HCatSchema subSchema = null; + + // populated if column is Map type + Type mapKeyType = null; + + private String typeString = null; + + @SuppressWarnings("unused") + private HCatFieldSchema() { + // preventing empty ctor from being callable + } + + /** + * Returns type of the field + * @return type of the field + */ + public Type getType() { + return type; + } + + /** + * Returns category of the field + * @return category of the field + */ + public Category getCategory() { + return category; + } + + /** + * Returns name of the field + * @return name of the field + */ + public String getName() { + return fieldName; + } + + public String getComment() { + return comment; + } + + /** + * Constructor constructing a primitive datatype HCatFieldSchema + * @param fieldName Name of the primitive field + * @param type Type of the primitive field + * @throws HCatException if call made on non-primitive types + */ + public HCatFieldSchema(String fieldName, Type type, String comment) throws HCatException { + assertTypeInCategory(type, Category.PRIMITIVE, fieldName); + this.fieldName = fieldName; + this.type = type; + this.category = Category.PRIMITIVE; + this.comment = comment; + } + + /** + * Constructor for constructing a ARRAY type or STRUCT type HCatFieldSchema, passing type and subschema + * @param fieldName Name of the array or struct field + * @param type Type of the field - either Type.ARRAY or Type.STRUCT + * @param subSchema - subschema of the struct, or element schema of the elements in the array + * @throws HCatException if call made on Primitive or Map types + */ + public HCatFieldSchema(String fieldName, Type type, HCatSchema subSchema, String comment) throws HCatException { + assertTypeNotInCategory(type, Category.PRIMITIVE); + assertTypeNotInCategory(type, Category.MAP); + this.fieldName = fieldName; + this.type = type; + this.category = Category.fromType(type); + this.subSchema = subSchema; + if (type == Type.ARRAY) { + this.subSchema.get(0).setName(null); + } + this.comment = comment; + } + + private void setName(String name) { + this.fieldName = name; + } + + /** + * Constructor for constructing a MAP type HCatFieldSchema, passing type of key and value + * @param fieldName Name of the array or struct field + * @param type Type of the field - must be Type.MAP + * @param mapKeyType - key type of the Map + * @param mapValueSchema - subschema of the value of the Map + * @throws HCatException if call made on non-Map types + */ + public HCatFieldSchema(String fieldName, Type type, Type mapKeyType, HCatSchema mapValueSchema, String comment) throws HCatException { + assertTypeInCategory(type, Category.MAP, fieldName); + assertTypeInCategory(mapKeyType, Category.PRIMITIVE, fieldName); + this.fieldName = fieldName; + this.type = Type.MAP; + this.category = Category.MAP; + this.mapKeyType = mapKeyType; + this.subSchema = mapValueSchema; + this.subSchema.get(0).setName(null); + this.comment = comment; + } + + public HCatSchema getStructSubSchema() throws HCatException { + assertTypeInCategory(this.type, Category.STRUCT, this.fieldName); + return subSchema; + } + + public HCatSchema getArrayElementSchema() throws HCatException { + assertTypeInCategory(this.type, Category.ARRAY, this.fieldName); + return subSchema; + } + + public Type getMapKeyType() throws HCatException { + assertTypeInCategory(this.type, Category.MAP, this.fieldName); + return mapKeyType; + } + + public HCatSchema getMapValueSchema() throws HCatException { + assertTypeInCategory(this.type, Category.MAP, this.fieldName); + return subSchema; + } + + private static void assertTypeInCategory(Type type, Category category, String fieldName) throws HCatException { + Category typeCategory = Category.fromType(type); + if (typeCategory != category) { + throw new HCatException("Type category mismatch. Expected " + category + " but type " + type + " in category " + typeCategory + " (field " + fieldName + ")"); + } + } + + private static void assertTypeNotInCategory(Type type, Category category) throws HCatException { + Category typeCategory = Category.fromType(type); + if (typeCategory == category) { + throw new HCatException("Type category mismatch. Expected type " + type + " not in category " + category + " but was so."); + } + } + + @Override + public String toString() { + return new ToStringBuilder(this) + .append("fieldName", fieldName) + .append("comment", comment) + .append("type", getTypeString()) + .append("category", category) + .toString(); + } + + public String getTypeString() { + if (typeString != null) { + return typeString; + } + + StringBuilder sb = new StringBuilder(); + if (Category.PRIMITIVE == category) { + sb.append(type); + } else if (Category.STRUCT == category) { + sb.append("struct<"); + sb.append(subSchema.getSchemaAsTypeString()); + sb.append(">"); + } else if (Category.ARRAY == category) { + sb.append("array<"); + sb.append(subSchema.getSchemaAsTypeString()); + sb.append(">"); + } else if (Category.MAP == category) { + sb.append("map<"); + sb.append(mapKeyType); + sb.append(","); + sb.append(subSchema.getSchemaAsTypeString()); + sb.append(">"); + } + return (typeString = sb.toString().toLowerCase()); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (!(obj instanceof HCatFieldSchema)) { + return false; + } + HCatFieldSchema other = (HCatFieldSchema) obj; + if (category != other.category) { + return false; + } + if (fieldName == null) { + if (other.fieldName != null) { + return false; + } + } else if (!fieldName.equals(other.fieldName)) { + return false; + } + if (this.getTypeString() == null) { + if (other.getTypeString() != null) { + return false; + } + } else if (!this.getTypeString().equals(other.getTypeString())) { + return false; + } + return true; + } + +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/data/schema/HCatSchema.java hcatalog/core/src/main/java/org/apache/hcatalog/data/schema/HCatSchema.java new file mode 100644 index 0000000..c053fff --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/data/schema/HCatSchema.java @@ -0,0 +1,184 @@ +/** + * 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.hcatalog.data.schema; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.hcatalog.common.HCatException; + +/** + * HCatSchema. This class is NOT thread-safe. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.data.schema.HCatSchema} instead + */ +public class HCatSchema implements Serializable { + + private static final long serialVersionUID = 1L; + + private final List fieldSchemas; + private final Map fieldPositionMap; + private final List fieldNames; + + /** + * + * @param fieldSchemas is now owned by HCatSchema. Any subsequent modifications + * on fieldSchemas won't get reflected in HCatSchema. Each fieldSchema's name + * in the list must be unique, otherwise throws IllegalArgumentException. + */ + public HCatSchema(final List fieldSchemas) { + this.fieldSchemas = new ArrayList(fieldSchemas); + int idx = 0; + fieldPositionMap = new HashMap(); + fieldNames = new ArrayList(); + for (HCatFieldSchema field : fieldSchemas) { + if (field == null) + throw new IllegalArgumentException("Field cannot be null"); + + String fieldName = field.getName(); + if (fieldPositionMap.containsKey(fieldName)) + throw new IllegalArgumentException("Field named " + fieldName + + " already exists"); + fieldPositionMap.put(fieldName, idx); + fieldNames.add(fieldName); + idx++; + } + } + + public void append(final HCatFieldSchema hfs) throws HCatException { + if (hfs == null) + throw new HCatException("Attempt to append null HCatFieldSchema in HCatSchema."); + + String fieldName = hfs.getName(); + if (fieldPositionMap.containsKey(fieldName)) + throw new HCatException("Attempt to append HCatFieldSchema with already " + + "existing name: " + fieldName + "."); + + this.fieldSchemas.add(hfs); + this.fieldNames.add(fieldName); + this.fieldPositionMap.put(fieldName, this.size() - 1); + } + + /** + * Users are not allowed to modify the list directly, since HCatSchema + * maintains internal state. Use append/remove to modify the schema. + */ + public List getFields() { + return Collections.unmodifiableList(this.fieldSchemas); + } + + /** + * @param fieldName + * @return the index of field named fieldName in Schema. If field is not + * present, returns null. + */ + public Integer getPosition(String fieldName) { + return fieldPositionMap.get(fieldName); + } + + public HCatFieldSchema get(String fieldName) throws HCatException { + return get(getPosition(fieldName)); + } + + public List getFieldNames() { + return this.fieldNames; + } + + public HCatFieldSchema get(int position) { + return fieldSchemas.get(position); + } + + public int size() { + return fieldSchemas.size(); + } + + public void remove(final HCatFieldSchema hcatFieldSchema) throws HCatException { + + if (!fieldSchemas.contains(hcatFieldSchema)) { + throw new HCatException("Attempt to delete a non-existent column from HCat Schema: " + hcatFieldSchema); + } + + fieldSchemas.remove(hcatFieldSchema); + fieldPositionMap.remove(hcatFieldSchema); + fieldNames.remove(hcatFieldSchema.getName()); + } + + @Override + public String toString() { + boolean first = true; + StringBuilder sb = new StringBuilder(); + for (HCatFieldSchema hfs : fieldSchemas) { + if (!first) { + sb.append(","); + } else { + first = false; + } + if (hfs.getName() != null) { + sb.append(hfs.getName()); + sb.append(":"); + } + sb.append(hfs.toString()); + } + return sb.toString(); + } + + public String getSchemaAsTypeString() { + boolean first = true; + StringBuilder sb = new StringBuilder(); + for (HCatFieldSchema hfs : fieldSchemas) { + if (!first) { + sb.append(","); + } else { + first = false; + } + if (hfs.getName() != null) { + sb.append(hfs.getName()); + sb.append(":"); + } + sb.append(hfs.getTypeString()); + } + return sb.toString(); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (!(obj instanceof HCatSchema)) { + return false; + } + HCatSchema other = (HCatSchema) obj; + if (!this.getFields().equals(other.getFields())) { + return false; + } + return true; + } + + @Override + public int hashCode() { + return toString().hashCode(); + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/data/schema/HCatSchemaUtils.java hcatalog/core/src/main/java/org/apache/hcatalog/data/schema/HCatSchemaUtils.java new file mode 100644 index 0000000..4aad89f --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/data/schema/HCatSchemaUtils.java @@ -0,0 +1,246 @@ +/** + * 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.hcatalog.data.schema; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.Schema; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category; +import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; +import org.apache.hcatalog.common.HCatConstants; +import org.apache.hcatalog.common.HCatContext; +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.data.schema.HCatFieldSchema.Type; + + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.data.schema.HCatSchemaUtils} instead + */ +public class HCatSchemaUtils { + + public static CollectionBuilder getStructSchemaBuilder() { + return new CollectionBuilder(); + } + + public static CollectionBuilder getListSchemaBuilder() { + return new CollectionBuilder(); + } + + public static MapBuilder getMapSchemaBuilder() { + return new MapBuilder(); + } + + + public static abstract class HCatSchemaBuilder { + public abstract HCatSchema build() throws HCatException; + } + + public static class CollectionBuilder extends HCatSchemaBuilder { // for STRUCTS(multiple-add-calls) and LISTS(single-add-call) + List fieldSchemas = null; + + CollectionBuilder() { + fieldSchemas = new ArrayList(); + } + + public CollectionBuilder addField(FieldSchema fieldSchema) throws HCatException { + return this.addField(getHCatFieldSchema(fieldSchema)); + } + + public CollectionBuilder addField(HCatFieldSchema fieldColumnSchema) { + fieldSchemas.add(fieldColumnSchema); + return this; + } + + @Override + public HCatSchema build() throws HCatException { + return new HCatSchema(fieldSchemas); + } + + } + + public static class MapBuilder extends HCatSchemaBuilder { + + Type keyType = null; + HCatSchema valueSchema = null; + + @Override + public HCatSchema build() throws HCatException { + List fslist = new ArrayList(); + fslist.add(new HCatFieldSchema(null, Type.MAP, keyType, valueSchema, null)); + return new HCatSchema(fslist); + } + + public MapBuilder withValueSchema(HCatSchema valueSchema) { + this.valueSchema = valueSchema; + return this; + } + + public MapBuilder withKeyType(Type keyType) { + this.keyType = keyType; + return this; + } + + } + + + /** + * Convert a HCatFieldSchema to a FieldSchema + * @param fs FieldSchema to convert + * @return HCatFieldSchema representation of FieldSchema + * @throws HCatException + */ + public static HCatFieldSchema getHCatFieldSchema(FieldSchema fs) throws HCatException { + String fieldName = fs.getName(); + TypeInfo baseTypeInfo = TypeInfoUtils.getTypeInfoFromTypeString(fs.getType()); + return getHCatFieldSchema(fieldName, baseTypeInfo); + } + + private static HCatFieldSchema getHCatFieldSchema(String fieldName, TypeInfo fieldTypeInfo) throws HCatException { + Category typeCategory = fieldTypeInfo.getCategory(); + HCatFieldSchema hCatFieldSchema; + if (Category.PRIMITIVE == typeCategory) { + hCatFieldSchema = new HCatFieldSchema(fieldName, getPrimitiveHType(fieldTypeInfo), null); + } else if (Category.STRUCT == typeCategory) { + HCatSchema subSchema = constructHCatSchema((StructTypeInfo) fieldTypeInfo); + hCatFieldSchema = new HCatFieldSchema(fieldName, HCatFieldSchema.Type.STRUCT, subSchema, null); + } else if (Category.LIST == typeCategory) { + HCatSchema subSchema = getHCatSchema(((ListTypeInfo) fieldTypeInfo).getListElementTypeInfo()); + hCatFieldSchema = new HCatFieldSchema(fieldName, HCatFieldSchema.Type.ARRAY, subSchema, null); + } else if (Category.MAP == typeCategory) { + HCatFieldSchema.Type mapKeyType = getPrimitiveHType(((MapTypeInfo) fieldTypeInfo).getMapKeyTypeInfo()); + HCatSchema subSchema = getHCatSchema(((MapTypeInfo) fieldTypeInfo).getMapValueTypeInfo()); + hCatFieldSchema = new HCatFieldSchema(fieldName, HCatFieldSchema.Type.MAP, mapKeyType, subSchema, null); + } else { + throw new TypeNotPresentException(fieldTypeInfo.getTypeName(), null); + } + return hCatFieldSchema; + } + + private static Type getPrimitiveHType(TypeInfo basePrimitiveTypeInfo) { + switch (((PrimitiveTypeInfo) basePrimitiveTypeInfo).getPrimitiveCategory()) { + case BOOLEAN: + return (HCatContext.INSTANCE.getConf().isPresent() && + HCatContext.INSTANCE.getConf().get().getBoolean( + HCatConstants.HCAT_DATA_CONVERT_BOOLEAN_TO_INTEGER, + HCatConstants.HCAT_DATA_CONVERT_BOOLEAN_TO_INTEGER_DEFAULT)) ? + Type.INT : Type.BOOLEAN; + case BYTE: + return (HCatContext.INSTANCE.getConf().isPresent() && + HCatContext.INSTANCE.getConf().get().getBoolean( + HCatConstants.HCAT_DATA_TINY_SMALL_INT_PROMOTION, + HCatConstants.HCAT_DATA_TINY_SMALL_INT_PROMOTION_DEFAULT)) ? + Type.INT : Type.TINYINT; + case DOUBLE: + return Type.DOUBLE; + case FLOAT: + return Type.FLOAT; + case INT: + return Type.INT; + case LONG: + return Type.BIGINT; + case SHORT: + return (HCatContext.INSTANCE.getConf().isPresent() && + HCatContext.INSTANCE.getConf().get().getBoolean( + HCatConstants.HCAT_DATA_TINY_SMALL_INT_PROMOTION, + HCatConstants.HCAT_DATA_TINY_SMALL_INT_PROMOTION_DEFAULT)) ? + Type.INT : Type.SMALLINT; + case STRING: + return Type.STRING; + case BINARY: + return Type.BINARY; + default: + throw new TypeNotPresentException(((PrimitiveTypeInfo) basePrimitiveTypeInfo).getTypeName(), null); + } + } + + public static HCatSchema getHCatSchema(Schema schema) throws HCatException { + return getHCatSchema(schema.getFieldSchemas()); + } + + public static HCatSchema getHCatSchema(List fslist) throws HCatException { + CollectionBuilder builder = getStructSchemaBuilder(); + for (FieldSchema fieldSchema : fslist) { + builder.addField(fieldSchema); + } + return builder.build(); + } + + private static HCatSchema constructHCatSchema(StructTypeInfo stypeInfo) throws HCatException { + CollectionBuilder builder = getStructSchemaBuilder(); + for (String fieldName : ((StructTypeInfo) stypeInfo).getAllStructFieldNames()) { + builder.addField(getHCatFieldSchema(fieldName, ((StructTypeInfo) stypeInfo).getStructFieldTypeInfo(fieldName))); + } + return builder.build(); + } + + public static HCatSchema getHCatSchema(TypeInfo typeInfo) throws HCatException { + Category typeCategory = typeInfo.getCategory(); + HCatSchema hCatSchema; + if (Category.PRIMITIVE == typeCategory) { + hCatSchema = getStructSchemaBuilder().addField(new HCatFieldSchema(null, getPrimitiveHType(typeInfo), null)).build(); + } else if (Category.STRUCT == typeCategory) { + HCatSchema subSchema = constructHCatSchema((StructTypeInfo) typeInfo); + hCatSchema = getStructSchemaBuilder().addField(new HCatFieldSchema(null, Type.STRUCT, subSchema, null)).build(); + } else if (Category.LIST == typeCategory) { + CollectionBuilder builder = getListSchemaBuilder(); + builder.addField(getHCatFieldSchema(null, ((ListTypeInfo) typeInfo).getListElementTypeInfo())); + hCatSchema = new HCatSchema(Arrays.asList(new HCatFieldSchema("", Type.ARRAY, builder.build(), ""))); + } else if (Category.MAP == typeCategory) { + HCatFieldSchema.Type mapKeyType = getPrimitiveHType(((MapTypeInfo) typeInfo).getMapKeyTypeInfo()); + HCatSchema subSchema = getHCatSchema(((MapTypeInfo) typeInfo).getMapValueTypeInfo()); + MapBuilder builder = getMapSchemaBuilder(); + hCatSchema = builder.withKeyType(mapKeyType).withValueSchema(subSchema).build(); + } else { + throw new TypeNotPresentException(typeInfo.getTypeName(), null); + } + return hCatSchema; + } + + public static HCatSchema getHCatSchemaFromTypeString(String typeString) throws HCatException { + return getHCatSchema(TypeInfoUtils.getTypeInfoFromTypeString(typeString)); + } + + public static HCatSchema getHCatSchema(String schemaString) throws HCatException { + if ((schemaString == null) || (schemaString.trim().isEmpty())) { + return new HCatSchema(new ArrayList()); // empty HSchema construct + } + HCatSchema outerSchema = getHCatSchemaFromTypeString("struct<" + schemaString + ">"); + return outerSchema.get(0).getStructSubSchema(); + } + + public static FieldSchema getFieldSchema(HCatFieldSchema hcatFieldSchema) { + return new FieldSchema(hcatFieldSchema.getName(), hcatFieldSchema.getTypeString(), hcatFieldSchema.getComment()); + } + + public static List getFieldSchemas(List hcatFieldSchemas) { + List lfs = new ArrayList(); + for (HCatFieldSchema hfs : hcatFieldSchemas) { + lfs.add(getFieldSchema(hfs)); + } + return lfs; + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/DataTransferFactory.java hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/DataTransferFactory.java new file mode 100644 index 0000000..8243083 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/DataTransferFactory.java @@ -0,0 +1,137 @@ +/** + * 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.hcatalog.data.transfer; + +import java.util.Map; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.mapreduce.InputSplit; +import org.apache.hcatalog.data.transfer.impl.HCatInputFormatReader; +import org.apache.hcatalog.data.transfer.impl.HCatOutputFormatWriter; +import org.apache.hcatalog.data.transfer.state.DefaultStateProvider; +import org.apache.hcatalog.data.transfer.state.StateProvider; + +/** + * Use this factory to get instances of {@link HCatReader} or {@link HCatWriter} + * at master and slave nodes. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.data.transfer.DataTransferFactory} instead + */ + +public class DataTransferFactory { + + /** + * This should be called once from master node to obtain an instance of + * {@link HCatReader}. + * + * @param re + * ReadEntity built using {@link ReadEntity.Builder} + * @param config + * any configuration which master node wants to pass to HCatalog + * @return {@link HCatReader} + */ + public static HCatReader getHCatReader(final ReadEntity re, + final Map config) { + // In future, this may examine ReadEntity and/or config to return + // appropriate HCatReader + return new HCatInputFormatReader(re, config); + } + + /** + * This should only be called once from every slave node to obtain an instance + * of {@link HCatReader}. + * + * @param split + * input split obtained at master node + * @param config + * configuration obtained at master node + * @return {@link HCatReader} + */ + public static HCatReader getHCatReader(final InputSplit split, + final Configuration config) { + // In future, this may examine config to return appropriate HCatReader + return getHCatReader(split, config, DefaultStateProvider.get()); + } + + /** + * This should only be called once from every slave node to obtain an instance + * of {@link HCatReader}. This should be called if an external system has some + * state to provide to HCatalog. + * + * @param split + * input split obtained at master node + * @param config + * configuration obtained at master node + * @param sp + * {@link StateProvider} + * @return {@link HCatReader} + */ + public static HCatReader getHCatReader(final InputSplit split, + final Configuration config, StateProvider sp) { + // In future, this may examine config to return appropriate HCatReader + return new HCatInputFormatReader(split, config, sp); + } + + /** + * This should be called at master node to obtain an instance of + * {@link HCatWriter}. + * + * @param we + * WriteEntity built using {@link WriteEntity.Builder} + * @param config + * any configuration which master wants to pass to HCatalog + * @return {@link HCatWriter} + */ + public static HCatWriter getHCatWriter(final WriteEntity we, + final Map config) { + // In future, this may examine WriteEntity and/or config to return + // appropriate HCatWriter + return new HCatOutputFormatWriter(we, config); + } + + /** + * This should be called at slave nodes to obtain an instance of + * {@link HCatWriter}. + * + * @param cntxt + * {@link WriterContext} obtained at master node + * @return {@link HCatWriter} + */ + public static HCatWriter getHCatWriter(final WriterContext cntxt) { + // In future, this may examine context to return appropriate HCatWriter + return getHCatWriter(cntxt, DefaultStateProvider.get()); + } + + /** + * This should be called at slave nodes to obtain an instance of + * {@link HCatWriter}. If an external system has some mechanism for providing + * state to HCatalog, this constructor can be used. + * + * @param cntxt + * {@link WriterContext} obtained at master node + * @param sp + * {@link StateProvider} + * @return {@link HCatWriter} + */ + public static HCatWriter getHCatWriter(final WriterContext cntxt, + final StateProvider sp) { + // In future, this may examine context to return appropriate HCatWriter + return new HCatOutputFormatWriter(cntxt.getConf(), sp); + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/EntityBase.java hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/EntityBase.java new file mode 100644 index 0000000..7175c50 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/EntityBase.java @@ -0,0 +1,61 @@ +/** + * 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.hcatalog.data.transfer; + +import java.util.Map; + +/** + * This is a base class for + * {@link ReadEntity.Builder} / {@link WriteEntity.Builder}. + * Many fields in them are common, so this class + * contains the common fields. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.data.transfer.EntityBase} instead + */ + +abstract class EntityBase { + + String region; + String tableName; + String dbName; + Map partitionKVs; + + /** + * Common methods for {@link ReadEntity} and {@link WriteEntity} + */ + + abstract static class Entity extends EntityBase { + + public String getRegion() { + return region; + } + + public String getTableName() { + return tableName; + } + + public String getDbName() { + return dbName; + } + + public Map getPartitionKVs() { + return partitionKVs; + } + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/HCatReader.java hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/HCatReader.java new file mode 100644 index 0000000..54550ec --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/HCatReader.java @@ -0,0 +1,103 @@ +/** + * 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.hcatalog.data.transfer; + +import java.util.Iterator; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.data.HCatRecord; +import org.apache.hcatalog.data.transfer.state.StateProvider; + +/** + * This abstract class is internal to HCatalog and abstracts away the notion of + * underlying system from which reads will be done. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.data.transfer.HCatReader} instead + */ + +public abstract class HCatReader { + + /** + * This should be called at master node to obtain {@link ReaderContext} which + * then should be serialized and sent to slave nodes. + * + * @return {@link ReaderContext} + * @throws HCatException + */ + public abstract ReaderContext prepareRead() throws HCatException; + + /** + * This should be called at slave nodes to read {@link HCatRecord}s + * + * @return {@link Iterator} of {@link HCatRecord} + * @throws HCatException + */ + public abstract Iterator read() throws HCatException; + + /** + * This constructor will be invoked by {@link DataTransferFactory} at master + * node. Don't use this constructor. Instead, use {@link DataTransferFactory} + * + * @param re + * @param config + */ + protected HCatReader(final ReadEntity re, final Map config) { + this(config); + this.re = re; + } + + /** + * This constructor will be invoked by {@link DataTransferFactory} at slave + * nodes. Don't use this constructor. Instead, use {@link DataTransferFactory} + * + * @param config + * @param sp + */ + + protected HCatReader(final Configuration config, StateProvider sp) { + this.conf = config; + this.sp = sp; + } + + protected ReadEntity re; // This will be null at slaves. + protected Configuration conf; + protected ReaderContext info; + protected StateProvider sp; // This will be null at master. + + private HCatReader(final Map config) { + Configuration conf = new Configuration(); + if (null != config) { + for (Entry kv : config.entrySet()) { + conf.set(kv.getKey(), kv.getValue()); + } + } + this.conf = conf; + } + + public Configuration getConf() { + if (null == conf) { + throw new IllegalStateException( + "HCatReader is not constructed correctly."); + } + return conf; + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/HCatWriter.java hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/HCatWriter.java new file mode 100644 index 0000000..c4ea928 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/HCatWriter.java @@ -0,0 +1,115 @@ +/** + * 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.hcatalog.data.transfer; + +import java.util.Iterator; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.data.HCatRecord; +import org.apache.hcatalog.data.transfer.state.StateProvider; + +/** + * This abstraction is internal to HCatalog. This is to facilitate writing to + * HCatalog from external systems. Don't try to instantiate this directly. + * Instead, use {@link DataTransferFactory} + * @deprecated Use/modify {@link org.apache.hive.hcatalog.data.transfer.HCatWriter} instead + */ + +public abstract class HCatWriter { + + protected Configuration conf; + protected WriteEntity we; // This will be null at slave nodes. + protected WriterContext info; + protected StateProvider sp; + + /** + * External system should invoke this method exactly once from a master node. + * + * @return {@link WriterContext} This should be serialized and sent to slave + * nodes to construct HCatWriter there. + * @throws HCatException + */ + public abstract WriterContext prepareWrite() throws HCatException; + + /** + * This method should be used at slave needs to perform writes. + * + * @param recordItr + * {@link Iterator} records to be written into HCatalog. + * @throws {@link HCatException} + */ + public abstract void write(final Iterator recordItr) + throws HCatException; + + /** + * This method should be called at master node. Primary purpose of this is to + * do metadata commit. + * + * @throws {@link HCatException} + */ + public abstract void commit(final WriterContext context) throws HCatException; + + /** + * This method should be called at master node. Primary purpose of this is to + * do cleanups in case of failures. + * + * @throws {@link HCatException} * + */ + public abstract void abort(final WriterContext context) throws HCatException; + + /** + * This constructor will be used at master node + * + * @param we + * WriteEntity defines where in storage records should be written to. + * @param config + * Any configuration which external system wants to communicate to + * HCatalog for performing writes. + */ + protected HCatWriter(final WriteEntity we, final Map config) { + this(config); + this.we = we; + } + + /** + * This constructor will be used at slave nodes. + * + * @param config + */ + protected HCatWriter(final Configuration config, final StateProvider sp) { + this.conf = config; + this.sp = sp; + } + + private HCatWriter(final Map config) { + Configuration conf = new Configuration(); + if (config != null) { + // user is providing config, so it could be null. + for (Entry kv : config.entrySet()) { + conf.set(kv.getKey(), kv.getValue()); + } + } + + this.conf = conf; + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/ReadEntity.java hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/ReadEntity.java new file mode 100644 index 0000000..b72cf73 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/ReadEntity.java @@ -0,0 +1,92 @@ +/** + * 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.hcatalog.data.transfer; + +import java.util.Map; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.data.transfer.ReadEntity} instead + */ +public class ReadEntity extends EntityBase.Entity { + + private String filterString; + + /** + * Don't instantiate {@link ReadEntity} directly. Use, + * {@link ReadEntity.Builder} instead. + * + */ + private ReadEntity() { + // Not allowed + } + + private ReadEntity(Builder builder) { + + this.region = builder.region; + this.dbName = builder.dbName; + this.tableName = builder.tableName; + this.partitionKVs = builder.partitionKVs; + this.filterString = builder.filterString; + } + + public String getFilterString() { + return this.filterString; + } + + /** + * This class should be used to build {@link ReadEntity}. It follows builder + * pattern, letting you build your {@link ReadEntity} with whatever level of + * detail you want. + * + */ + public static class Builder extends EntityBase { + + private String filterString; + + public Builder withRegion(final String region) { + this.region = region; + return this; + } + + public Builder withDatabase(final String dbName) { + this.dbName = dbName; + return this; + } + + public Builder withTable(final String tblName) { + this.tableName = tblName; + return this; + } + + public Builder withPartition(final Map partKVs) { + this.partitionKVs = partKVs; + return this; + } + + public Builder withFilter(String filterString) { + this.filterString = filterString; + return this; + } + + public ReadEntity build() { + return new ReadEntity(this); + } + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/ReaderContext.java hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/ReaderContext.java new file mode 100644 index 0000000..08be77f --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/ReaderContext.java @@ -0,0 +1,90 @@ +/** + * 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.hcatalog.data.transfer; + +import java.io.Externalizable; +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.conf.Configurable; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.mapreduce.InputSplit; +import org.apache.hcatalog.mapreduce.HCatSplit; + +/** + * This class will contain information of different {@link InputSplit} obtained + * at master node and configuration. This class implements + * {@link Externalizable} so it can be serialized using standard java + * mechanisms. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.data.transfer.ReaderContext} instead + */ +public class ReaderContext implements Externalizable, Configurable { + + private static final long serialVersionUID = -2656468331739574367L; + private List splits; + private Configuration conf; + + public ReaderContext() { + this.splits = new ArrayList(); + this.conf = new Configuration(); + } + + public void setInputSplits(final List splits) { + this.splits = splits; + } + + public List getSplits() { + return splits; + } + + @Override + public Configuration getConf() { + return conf; + } + + @Override + public void setConf(final Configuration config) { + conf = config; + } + + @Override + public void writeExternal(ObjectOutput out) throws IOException { + conf.write(out); + out.writeInt(splits.size()); + for (InputSplit split : splits) { + ((HCatSplit) split).write(out); + } + } + + @Override + public void readExternal(ObjectInput in) throws IOException, + ClassNotFoundException { + conf.readFields(in); + int numOfSplits = in.readInt(); + for (int i = 0; i < numOfSplits; i++) { + HCatSplit split = new HCatSplit(); + split.readFields(in); + splits.add(split); + } + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/WriteEntity.java hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/WriteEntity.java new file mode 100644 index 0000000..9940e96 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/WriteEntity.java @@ -0,0 +1,78 @@ +/** + * 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.hcatalog.data.transfer; + +import java.util.Map; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.data.transfer.WriteEntity} instead + */ +public class WriteEntity extends EntityBase.Entity { + + /** + * Don't instantiate {@link WriteEntity} directly. Use, {@link Builder} to + * build {@link WriteEntity}. + */ + + private WriteEntity() { + // Not allowed. + } + + private WriteEntity(Builder builder) { + this.region = builder.region; + this.dbName = builder.dbName; + this.tableName = builder.tableName; + this.partitionKVs = builder.partitionKVs; + } + + /** + * This class should be used to build {@link WriteEntity}. It follows builder + * pattern, letting you build your {@link WriteEntity} with whatever level of + * detail you want. + * + */ + public static class Builder extends EntityBase { + + public Builder withRegion(final String region) { + this.region = region; + return this; + } + + public Builder withDatabase(final String dbName) { + this.dbName = dbName; + return this; + } + + public Builder withTable(final String tblName) { + this.tableName = tblName; + return this; + } + + public Builder withPartition(final Map partKVs) { + this.partitionKVs = partKVs; + return this; + } + + public WriteEntity build() { + return new WriteEntity(this); + } + + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/WriterContext.java hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/WriterContext.java new file mode 100644 index 0000000..9cd9a6d --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/WriterContext.java @@ -0,0 +1,66 @@ +/** + * 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.hcatalog.data.transfer; + +import java.io.Externalizable; +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; + +import org.apache.hadoop.conf.Configurable; +import org.apache.hadoop.conf.Configuration; + +/** + * This contains information obtained at master node to help prepare slave nodes + * for writer. This class implements {@link Externalizable} so it can be + * serialized using standard java mechanisms. Master should serialize it and + * make it available to slaves to prepare for writes. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.data.transfer.WriterContext} instead + */ +public class WriterContext implements Externalizable, Configurable { + + private static final long serialVersionUID = -5899374262971611840L; + private Configuration conf; + + public WriterContext() { + conf = new Configuration(); + } + + @Override + public Configuration getConf() { + return conf; + } + + @Override + public void setConf(final Configuration config) { + this.conf = config; + } + + @Override + public void writeExternal(ObjectOutput out) throws IOException { + conf.write(out); + } + + @Override + public void readExternal(ObjectInput in) throws IOException, + ClassNotFoundException { + conf.readFields(in); + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/impl/HCatInputFormatReader.java hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/impl/HCatInputFormatReader.java new file mode 100644 index 0000000..01b2f7d --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/impl/HCatInputFormatReader.java @@ -0,0 +1,137 @@ +/** + * 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.hcatalog.data.transfer.impl; + +import java.io.IOException; +import java.util.Iterator; +import java.util.Map; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.WritableComparable; +import org.apache.hadoop.mapreduce.InputSplit; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.RecordReader; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.TaskAttemptID; +import org.apache.hcatalog.common.ErrorType; +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.data.HCatRecord; +import org.apache.hcatalog.data.transfer.HCatReader; +import org.apache.hcatalog.data.transfer.ReadEntity; +import org.apache.hcatalog.data.transfer.ReaderContext; +import org.apache.hcatalog.data.transfer.state.StateProvider; +import org.apache.hcatalog.mapreduce.HCatInputFormat; +import org.apache.hadoop.hive.shims.ShimLoader; + +/** + * This reader reads via {@link HCatInputFormat} + * @deprecated Use/modify {@link org.apache.hive.hcatalog.data.transfer.impl.HCatInputFormatReader} instead + */ +public class HCatInputFormatReader extends HCatReader { + + private InputSplit split; + + public HCatInputFormatReader(InputSplit split, Configuration config, + StateProvider sp) { + super(config, sp); + this.split = split; + } + + public HCatInputFormatReader(ReadEntity info, Map config) { + super(info, config); + } + + @Override + public ReaderContext prepareRead() throws HCatException { + try { + Job job = new Job(conf); + HCatInputFormat hcif = HCatInputFormat.setInput( + job, re.getDbName(), re.getTableName()).setFilter(re.getFilterString()); + ReaderContext cntxt = new ReaderContext(); + cntxt.setInputSplits(hcif.getSplits( + ShimLoader.getHadoopShims().getHCatShim().createJobContext(job.getConfiguration(), null))); + cntxt.setConf(job.getConfiguration()); + return cntxt; + } catch (IOException e) { + throw new HCatException(ErrorType.ERROR_NOT_INITIALIZED, e); + } catch (InterruptedException e) { + throw new HCatException(ErrorType.ERROR_NOT_INITIALIZED, e); + } + } + + @Override + public Iterator read() throws HCatException { + + HCatInputFormat inpFmt = new HCatInputFormat(); + RecordReader rr; + try { + TaskAttemptContext cntxt = ShimLoader.getHadoopShims().getHCatShim().createTaskAttemptContext(conf, new TaskAttemptID()); + rr = inpFmt.createRecordReader(split, cntxt); + rr.initialize(split, cntxt); + } catch (IOException e) { + throw new HCatException(ErrorType.ERROR_NOT_INITIALIZED, e); + } catch (InterruptedException e) { + throw new HCatException(ErrorType.ERROR_NOT_INITIALIZED, e); + } + return new HCatRecordItr(rr); + } + + private static class HCatRecordItr implements Iterator { + + private RecordReader curRecReader; + + HCatRecordItr(RecordReader rr) { + curRecReader = rr; + } + + @Override + public boolean hasNext() { + try { + boolean retVal = curRecReader.nextKeyValue(); + if (retVal) { + return true; + } + // if its false, we need to close recordReader. + curRecReader.close(); + return false; + } catch (IOException e) { + throw new RuntimeException(e); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + + @Override + public HCatRecord next() { + try { + return curRecReader.getCurrentValue(); + } catch (IOException e) { + throw new RuntimeException(e); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + + @Override + public void remove() { + throw new UnsupportedOperationException("Not allowed"); + } + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/impl/HCatOutputFormatWriter.java hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/impl/HCatOutputFormatWriter.java new file mode 100644 index 0000000..d420416 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/impl/HCatOutputFormatWriter.java @@ -0,0 +1,161 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hcatalog.data.transfer.impl; + +import java.io.IOException; +import java.util.Iterator; +import java.util.Map; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.WritableComparable; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.JobStatus.State; +import org.apache.hadoop.mapreduce.OutputCommitter; +import org.apache.hadoop.mapreduce.RecordWriter; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.TaskAttemptID; +import org.apache.hcatalog.common.ErrorType; +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.data.HCatRecord; +import org.apache.hcatalog.data.transfer.HCatWriter; +import org.apache.hcatalog.data.transfer.WriteEntity; +import org.apache.hcatalog.data.transfer.WriterContext; +import org.apache.hcatalog.data.transfer.state.StateProvider; +import org.apache.hcatalog.mapreduce.HCatOutputFormat; +import org.apache.hcatalog.mapreduce.OutputJobInfo; +import org.apache.hadoop.hive.shims.ShimLoader; + +/** + * This writer writes via {@link HCatOutputFormat} + * @deprecated Use/modify {@link org.apache.hive.hcatalog.data.transfer.impl.HCatOutputFormatWriter} instead + */ +public class HCatOutputFormatWriter extends HCatWriter { + + public HCatOutputFormatWriter(WriteEntity we, Map config) { + super(we, config); + } + + public HCatOutputFormatWriter(Configuration config, StateProvider sp) { + super(config, sp); + } + + @Override + public WriterContext prepareWrite() throws HCatException { + OutputJobInfo jobInfo = OutputJobInfo.create(we.getDbName(), + we.getTableName(), we.getPartitionKVs()); + Job job; + try { + job = new Job(conf); + HCatOutputFormat.setOutput(job, jobInfo); + HCatOutputFormat.setSchema(job, HCatOutputFormat.getTableSchema(job)); + HCatOutputFormat outFormat = new HCatOutputFormat(); + outFormat.checkOutputSpecs(job); + outFormat.getOutputCommitter(ShimLoader.getHadoopShims().getHCatShim().createTaskAttemptContext( + job.getConfiguration(), ShimLoader.getHadoopShims().getHCatShim().createTaskAttemptID())).setupJob(job); + } catch (IOException e) { + throw new HCatException(ErrorType.ERROR_NOT_INITIALIZED, e); + } catch (InterruptedException e) { + throw new HCatException(ErrorType.ERROR_NOT_INITIALIZED, e); + } + WriterContext cntxt = new WriterContext(); + cntxt.setConf(job.getConfiguration()); + return cntxt; + } + + @Override + public void write(Iterator recordItr) throws HCatException { + + int id = sp.getId(); + setVarsInConf(id); + HCatOutputFormat outFormat = new HCatOutputFormat(); + TaskAttemptContext cntxt = ShimLoader.getHadoopShims().getHCatShim().createTaskAttemptContext( + conf, new TaskAttemptID(ShimLoader.getHadoopShims().getHCatShim().createTaskID(), id)); + OutputCommitter committer = null; + RecordWriter, HCatRecord> writer; + try { + committer = outFormat.getOutputCommitter(cntxt); + committer.setupTask(cntxt); + writer = outFormat.getRecordWriter(cntxt); + while (recordItr.hasNext()) { + HCatRecord rec = recordItr.next(); + writer.write(null, rec); + } + writer.close(cntxt); + if (committer.needsTaskCommit(cntxt)) { + committer.commitTask(cntxt); + } + } catch (IOException e) { + if (null != committer) { + try { + committer.abortTask(cntxt); + } catch (IOException e1) { + throw new HCatException(ErrorType.ERROR_INTERNAL_EXCEPTION, e1); + } + } + throw new HCatException("Failed while writing", e); + } catch (InterruptedException e) { + if (null != committer) { + try { + committer.abortTask(cntxt); + } catch (IOException e1) { + throw new HCatException(ErrorType.ERROR_INTERNAL_EXCEPTION, e1); + } + } + throw new HCatException("Failed while writing", e); + } + } + + @Override + public void commit(WriterContext context) throws HCatException { + try { + new HCatOutputFormat().getOutputCommitter(ShimLoader.getHadoopShims().getHCatShim().createTaskAttemptContext( + context.getConf(), ShimLoader.getHadoopShims().getHCatShim().createTaskAttemptID())) + .commitJob(ShimLoader.getHadoopShims().getHCatShim().createJobContext(context.getConf(), null)); + } catch (IOException e) { + throw new HCatException(ErrorType.ERROR_NOT_INITIALIZED, e); + } catch (InterruptedException e) { + throw new HCatException(ErrorType.ERROR_NOT_INITIALIZED, e); + } + } + + @Override + public void abort(WriterContext context) throws HCatException { + try { + new HCatOutputFormat().getOutputCommitter(ShimLoader.getHadoopShims().getHCatShim().createTaskAttemptContext( + context.getConf(), ShimLoader.getHadoopShims().getHCatShim().createTaskAttemptID())) + .abortJob(ShimLoader.getHadoopShims().getHCatShim().createJobContext(context.getConf(), null), State.FAILED); + } catch (IOException e) { + throw new HCatException(ErrorType.ERROR_NOT_INITIALIZED, e); + } catch (InterruptedException e) { + throw new HCatException(ErrorType.ERROR_NOT_INITIALIZED, e); + } + } + + private void setVarsInConf(int id) { + + // Following two config keys are required by FileOutputFormat to work + // correctly. + // In usual case of Hadoop, JobTracker will set these before launching + // tasks. + // Since there is no jobtracker here, we set it ourself. + conf.setInt("mapred.task.partition", id); + conf.set("mapred.task.id", "attempt__0000_r_000000_" + id); + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/state/DefaultStateProvider.java hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/state/DefaultStateProvider.java new file mode 100644 index 0000000..80bf6c0 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/state/DefaultStateProvider.java @@ -0,0 +1,51 @@ +/** + * 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.hcatalog.data.transfer.state; + +import java.text.NumberFormat; +import java.util.Random; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.data.transfer.state.DefaultStateProvider} instead + */ +public class DefaultStateProvider implements StateProvider { + + /** + * Default implementation. Here, ids are generated randomly. + */ + @Override + public int getId() { + + NumberFormat numberFormat = NumberFormat.getInstance(); + numberFormat.setMinimumIntegerDigits(5); + numberFormat.setGroupingUsed(false); + return Integer + .parseInt(numberFormat.format(Math.abs(new Random().nextInt()))); + } + + private static StateProvider sp; + + public static synchronized StateProvider get() { + if (null == sp) { + sp = new DefaultStateProvider(); + } + return sp; + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/state/StateProvider.java hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/state/StateProvider.java new file mode 100644 index 0000000..2aec2ab --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/data/transfer/state/StateProvider.java @@ -0,0 +1,36 @@ +/** + * 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.hcatalog.data.transfer.state; + +/** + * If external system wants to communicate any state to slaves, they can do so + * via this interface. One example of this in case of Map-Reduce is ids assigned + * by JobTracker to TaskTracker. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.data.transfer.state.StateProvider} instead + */ +public interface StateProvider { + + /** + * This method should return id assigned to slave node. + * + * @return id + */ + public int getId(); +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/har/HarOutputCommitterPostProcessor.java hcatalog/core/src/main/java/org/apache/hcatalog/har/HarOutputCommitterPostProcessor.java new file mode 100644 index 0000000..10ad2da --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/har/HarOutputCommitterPostProcessor.java @@ -0,0 +1,128 @@ +/** + * 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.hcatalog.har; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.tools.HadoopArchives; +import org.apache.hadoop.util.ToolRunner; +import org.apache.hcatalog.common.HCatConstants; +import org.apache.hcatalog.common.HCatException; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.har.HarOutputCommitterPostProcessor} instead + */ +public class HarOutputCommitterPostProcessor { + + boolean isEnabled = false; + + public boolean isEnabled() { + return isEnabled; + } + + public void setEnabled(boolean enabled) { + this.isEnabled = enabled; + } + + + public void exec(JobContext context, Partition partition, Path partPath) throws IOException { +// LOG.info("Archiving partition ["+partPath.toString()+"]"); + makeHar(context, partPath.toUri().toString(), harFile(partPath)); + partition.getParameters().put(hive_metastoreConstants.IS_ARCHIVED, "true"); + } + + public String harFile(Path ptnPath) throws IOException { + String harFile = ptnPath.toString().replaceFirst("/+$", "") + ".har"; +// LOG.info("har file : " + harFile); + return harFile; + } + + public String getParentFSPath(Path ptnPath) throws IOException { + return ptnPath.toUri().getPath().replaceFirst("/+$", ""); + } + + public String getProcessedLocation(Path ptnPath) throws IOException { + String harLocn = ("har://" + ptnPath.toUri().getPath()).replaceFirst("/+$", "") + ".har" + Path.SEPARATOR; +// LOG.info("har location : " + harLocn); + return harLocn; + } + + + /** + * Creates a har file from the contents of a given directory, using that as root. + * @param dir Directory to archive + * @param harFile The HAR file to create + */ + public static void makeHar(JobContext context, String dir, String harFile) throws IOException { +// Configuration conf = context.getConfiguration(); +// Credentials creds = context.getCredentials(); + +// HCatUtil.logAllTokens(LOG,context); + + int lastSep = harFile.lastIndexOf(Path.SEPARATOR_CHAR); + Path archivePath = new Path(harFile.substring(0, lastSep)); + final String[] args = { + "-archiveName", + harFile.substring(lastSep + 1, harFile.length()), + "-p", + dir, + "*", + archivePath.toString() + }; +// for (String arg : args){ +// LOG.info("Args to har : "+ arg); +// } + try { + Configuration newConf = new Configuration(); + FileSystem fs = archivePath.getFileSystem(newConf); + + String hadoopTokenFileLocationEnvSetting = System.getenv(HCatConstants.SYSENV_HADOOP_TOKEN_FILE_LOCATION); + if ((hadoopTokenFileLocationEnvSetting != null) && (!hadoopTokenFileLocationEnvSetting.isEmpty())) { + newConf.set(HCatConstants.CONF_MAPREDUCE_JOB_CREDENTIALS_BINARY, hadoopTokenFileLocationEnvSetting); +// LOG.info("System.getenv(\"HADOOP_TOKEN_FILE_LOCATION\") =["+ System.getenv("HADOOP_TOKEN_FILE_LOCATION")+"]"); + } +// for (FileStatus ds : fs.globStatus(new Path(dir, "*"))){ +// LOG.info("src : "+ds.getPath().toUri().toString()); +// } + + final HadoopArchives har = new HadoopArchives(newConf); + int rc = ToolRunner.run(har, args); + if (rc != 0) { + throw new Exception("Har returned error code " + rc); + } + +// for (FileStatus hs : fs.globStatus(new Path(harFile, "*"))){ +// LOG.info("dest : "+hs.getPath().toUri().toString()); +// } +// doHarCheck(fs,harFile); +// LOG.info("Nuking " + dir); + fs.delete(new Path(dir), true); + } catch (Exception e) { + throw new HCatException("Error creating Har [" + harFile + "] from [" + dir + "]", e); + } + } + +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/DefaultOutputCommitterContainer.java hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/DefaultOutputCommitterContainer.java new file mode 100644 index 0000000..89f94ed --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/DefaultOutputCommitterContainer.java @@ -0,0 +1,109 @@ +/** + * 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.hcatalog.mapreduce; + +import java.io.IOException; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hive.hcatalog.mapreduce.HCatMapRedUtil; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.JobStatus.State; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hcatalog.common.HCatConstants; +import org.apache.hcatalog.common.HCatUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Part of the DefaultOutput*Container classes + * See {@link DefaultOutputFormatContainer} for more information + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.DefaultOutputCommitterContainer} instead + */ +class DefaultOutputCommitterContainer extends OutputCommitterContainer { + + private static final Logger LOG = LoggerFactory.getLogger(DefaultOutputCommitterContainer.class); + + /** + * @param context current JobContext + * @param baseCommitter OutputCommitter to contain + * @throws IOException + */ + public DefaultOutputCommitterContainer(JobContext context, org.apache.hadoop.mapred.OutputCommitter baseCommitter) throws IOException { + super(context, baseCommitter); + } + + @Override + public void abortTask(TaskAttemptContext context) throws IOException { + getBaseOutputCommitter().abortTask(HCatMapRedUtil.createTaskAttemptContext(context)); + } + + @Override + public void commitTask(TaskAttemptContext context) throws IOException { + getBaseOutputCommitter().commitTask(HCatMapRedUtil.createTaskAttemptContext(context)); + } + + @Override + public boolean needsTaskCommit(TaskAttemptContext context) throws IOException { + return getBaseOutputCommitter().needsTaskCommit(HCatMapRedUtil.createTaskAttemptContext(context)); + } + + @Override + public void setupJob(JobContext context) throws IOException { + getBaseOutputCommitter().setupJob(HCatMapRedUtil.createJobContext(context)); + } + + @Override + public void setupTask(TaskAttemptContext context) throws IOException { + getBaseOutputCommitter().setupTask(HCatMapRedUtil.createTaskAttemptContext(context)); + } + + @Override + public void abortJob(JobContext jobContext, State state) throws IOException { + getBaseOutputCommitter().abortJob(HCatMapRedUtil.createJobContext(jobContext), state); + cleanupJob(jobContext); + } + + @Override + public void commitJob(JobContext jobContext) throws IOException { + getBaseOutputCommitter().commitJob(HCatMapRedUtil.createJobContext(jobContext)); + cleanupJob(jobContext); + } + + @Override + public void cleanupJob(JobContext context) throws IOException { + getBaseOutputCommitter().cleanupJob(HCatMapRedUtil.createJobContext(context)); + + //Cancel HCat and JobTracker tokens + HiveMetaStoreClient client = null; + try { + HiveConf hiveConf = HCatUtil.getHiveConf(context.getConfiguration()); + client = HCatUtil.getHiveClient(hiveConf); + String tokenStrForm = client.getTokenStrForm(); + if (tokenStrForm != null && context.getConfiguration().get(HCatConstants.HCAT_KEY_TOKEN_SIGNATURE) != null) { + client.cancelDelegationToken(tokenStrForm); + } + } catch (Exception e) { + LOG.warn("Failed to cancel delegation token", e); + } finally { + HCatUtil.closeHiveClientQuietly(client); + } + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/DefaultOutputFormatContainer.java hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/DefaultOutputFormatContainer.java new file mode 100644 index 0000000..df0c141 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/DefaultOutputFormatContainer.java @@ -0,0 +1,102 @@ +/** + * 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.hcatalog.mapreduce; + +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.io.WritableComparable; + +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.OutputCommitter; +import org.apache.hadoop.mapreduce.RecordWriter; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hcatalog.common.HCatUtil; +import org.apache.hcatalog.data.HCatRecord; + +import java.io.IOException; +import java.text.NumberFormat; + +/** + * Bare bones implementation of OutputFormatContainer. Does only the required + * tasks to work properly with HCatalog. HCatalog features which require a + * storage specific implementation are unsupported (ie partitioning). + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.DefaultOutputFormatContainer} instead + */ +class DefaultOutputFormatContainer extends OutputFormatContainer { + + private static final NumberFormat NUMBER_FORMAT = NumberFormat.getInstance(); + + static { + NUMBER_FORMAT.setMinimumIntegerDigits(5); + NUMBER_FORMAT.setGroupingUsed(false); + } + + public DefaultOutputFormatContainer(org.apache.hadoop.mapred.OutputFormat, Writable> of) { + super(of); + } + + static synchronized String getOutputName(int partition) { + return "part-" + NUMBER_FORMAT.format(partition); + } + + /** + * Get the record writer for the job. Uses the storagehandler's OutputFormat + * to get the record writer. + * @param context the information about the current task. + * @return a RecordWriter to write the output for the job. + * @throws IOException + */ + @Override + public RecordWriter, HCatRecord> + getRecordWriter(TaskAttemptContext context) throws IOException, InterruptedException { + String name = getOutputName(context.getTaskAttemptID().getTaskID().getId()); + return new DefaultRecordWriterContainer(context, + getBaseOutputFormat().getRecordWriter(null, new JobConf(context.getConfiguration()), name, InternalUtil.createReporter(context))); + } + + + /** + * Get the output committer for this output format. This is responsible + * for ensuring the output is committed correctly. + * @param context the task context + * @return an output committer + * @throws IOException + * @throws InterruptedException + */ + @Override + public OutputCommitter getOutputCommitter(TaskAttemptContext context) + throws IOException, InterruptedException { + return new DefaultOutputCommitterContainer(context, new JobConf(context.getConfiguration()).getOutputCommitter()); + } + + /** + * Check for validity of the output-specification for the job. + * @param context information about the job + * @throws IOException when output should not be attempted + */ + @Override + public void checkOutputSpecs(JobContext context) throws IOException, InterruptedException { + org.apache.hadoop.mapred.OutputFormat, ? super Writable> outputFormat = getBaseOutputFormat(); + JobConf jobConf = new JobConf(context.getConfiguration()); + outputFormat.checkOutputSpecs(null, jobConf); + HCatUtil.copyConf(jobConf, context.getConfiguration()); + } + +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/DefaultRecordWriterContainer.java hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/DefaultRecordWriterContainer.java new file mode 100644 index 0000000..a6c0eda --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/DefaultRecordWriterContainer.java @@ -0,0 +1,83 @@ +/** + * 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.hcatalog.mapreduce; + +import java.io.IOException; + +import org.apache.hadoop.hive.serde2.SerDe; +import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.io.WritableComparable; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.util.ReflectionUtils; +import org.apache.hcatalog.common.HCatUtil; +import org.apache.hcatalog.data.HCatRecord; + +/** + * Part of the DefaultOutput*Container classes + * See {@link DefaultOutputFormatContainer} for more information + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.DefaultRecordWriterContainer} instead + */ +class DefaultRecordWriterContainer extends RecordWriterContainer { + + private final HCatStorageHandler storageHandler; + private final SerDe serDe; + private final OutputJobInfo jobInfo; + private final ObjectInspector hcatRecordOI; + + /** + * @param context current JobContext + * @param baseRecordWriter RecordWriter to contain + * @throws IOException + * @throws InterruptedException + */ + public DefaultRecordWriterContainer(TaskAttemptContext context, + org.apache.hadoop.mapred.RecordWriter, ? super Writable> baseRecordWriter) throws IOException, InterruptedException { + super(context, baseRecordWriter); + jobInfo = HCatOutputFormat.getJobInfo(context); + storageHandler = HCatUtil.getStorageHandler(context.getConfiguration(), jobInfo.getTableInfo().getStorerInfo()); + HCatOutputFormat.configureOutputStorageHandler(context); + serDe = ReflectionUtils.newInstance(storageHandler.getSerDeClass(), context.getConfiguration()); + hcatRecordOI = InternalUtil.createStructObjectInspector(jobInfo.getOutputSchema()); + try { + InternalUtil.initializeOutputSerDe(serDe, context.getConfiguration(), jobInfo); + } catch (SerDeException e) { + throw new IOException("Failed to initialize SerDe", e); + } + } + + @Override + public void close(TaskAttemptContext context) throws IOException, + InterruptedException { + getBaseRecordWriter().close(InternalUtil.createReporter(context)); + } + + @Override + public void write(WritableComparable key, HCatRecord value) throws IOException, + InterruptedException { + try { + getBaseRecordWriter().write(null, serDe.serialize(value.getAll(), hcatRecordOI)); + } catch (SerDeException e) { + throw new IOException("Failed to serialize object", e); + } + } + +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/FileOutputCommitterContainer.java hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/FileOutputCommitterContainer.java new file mode 100644 index 0000000..383edaa --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/FileOutputCommitterContainer.java @@ -0,0 +1,747 @@ +/** + * 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.hcatalog.mapreduce; + +import java.io.IOException; +import java.net.URI; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hive.common.FileUtils; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.Warehouse; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.InvalidOperationException; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hive.hcatalog.mapreduce.HCatMapRedUtil; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.JobStatus.State; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hcatalog.common.ErrorType; +import org.apache.hcatalog.common.HCatConstants; +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.common.HCatUtil; +import org.apache.hcatalog.data.schema.HCatFieldSchema; +import org.apache.hcatalog.data.schema.HCatSchema; +import org.apache.hcatalog.data.schema.HCatSchemaUtils; +import org.apache.hcatalog.har.HarOutputCommitterPostProcessor; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Part of the FileOutput*Container classes + * See {@link FileOutputFormatContainer} for more information + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.FileOutputCommitterContainer} instead + */ +class FileOutputCommitterContainer extends OutputCommitterContainer { + + private static final String TEMP_DIR_NAME = "_temporary"; + private static final String LOGS_DIR_NAME = "_logs"; + + private static final Logger LOG = LoggerFactory.getLogger(FileOutputCommitterContainer.class); + private final boolean dynamicPartitioningUsed; + private boolean partitionsDiscovered; + + private Map> partitionsDiscoveredByPath; + private Map contextDiscoveredByPath; + private final HCatStorageHandler cachedStorageHandler; + + HarOutputCommitterPostProcessor harProcessor = new HarOutputCommitterPostProcessor(); + + private String ptnRootLocation = null; + + private OutputJobInfo jobInfo = null; + + /** + * @param context current JobContext + * @param baseCommitter OutputCommitter to contain + * @throws IOException + */ + public FileOutputCommitterContainer(JobContext context, + org.apache.hadoop.mapred.OutputCommitter baseCommitter) throws IOException { + super(context, baseCommitter); + jobInfo = HCatOutputFormat.getJobInfo(context); + dynamicPartitioningUsed = jobInfo.isDynamicPartitioningUsed(); + + this.partitionsDiscovered = !dynamicPartitioningUsed; + cachedStorageHandler = HCatUtil.getStorageHandler(context.getConfiguration(), jobInfo.getTableInfo().getStorerInfo()); + } + + @Override + public void abortTask(TaskAttemptContext context) throws IOException { + if (!dynamicPartitioningUsed) { + getBaseOutputCommitter().abortTask(HCatMapRedUtil.createTaskAttemptContext(context)); + } + } + + @Override + public void commitTask(TaskAttemptContext context) throws IOException { + if (!dynamicPartitioningUsed) { + //See HCATALOG-499 + FileOutputFormatContainer.setWorkOutputPath(context); + getBaseOutputCommitter().commitTask(HCatMapRedUtil.createTaskAttemptContext(context)); + } + } + + @Override + public boolean needsTaskCommit(TaskAttemptContext context) throws IOException { + if (!dynamicPartitioningUsed) { + return getBaseOutputCommitter().needsTaskCommit(HCatMapRedUtil.createTaskAttemptContext(context)); + } else { + // called explicitly through FileRecordWriterContainer.close() if dynamic - return false by default + return false; + } + } + + @Override + public void setupJob(JobContext context) throws IOException { + if (getBaseOutputCommitter() != null && !dynamicPartitioningUsed) { + getBaseOutputCommitter().setupJob(HCatMapRedUtil.createJobContext(context)); + } + // in dynamic usecase, called through FileRecordWriterContainer + } + + @Override + public void setupTask(TaskAttemptContext context) throws IOException { + if (!dynamicPartitioningUsed) { + getBaseOutputCommitter().setupTask(HCatMapRedUtil.createTaskAttemptContext(context)); + } + } + + @Override + public void abortJob(JobContext jobContext, State state) throws IOException { + try { + if (dynamicPartitioningUsed) { + discoverPartitions(jobContext); + } + org.apache.hadoop.mapred.JobContext mapRedJobContext = HCatMapRedUtil + .createJobContext(jobContext); + if (getBaseOutputCommitter() != null && !dynamicPartitioningUsed) { + getBaseOutputCommitter().abortJob(mapRedJobContext, state); + } else if (dynamicPartitioningUsed) { + for (JobContext currContext : contextDiscoveredByPath.values()) { + try { + new JobConf(currContext.getConfiguration()) + .getOutputCommitter().abortJob(currContext, + state); + } catch (Exception e) { + throw new IOException(e); + } + } + } + Path src; + OutputJobInfo jobInfo = HCatOutputFormat.getJobInfo(jobContext); + if (dynamicPartitioningUsed) { + src = new Path(getPartitionRootLocation(jobInfo.getLocation(), jobInfo.getTableInfo().getTable() + .getPartitionKeysSize())); + } else { + src = new Path(jobInfo.getLocation()); + } + FileSystem fs = src.getFileSystem(jobContext.getConfiguration()); + LOG.info("Job failed. Cleaning up temporary directory [{}].", src); + fs.delete(src, true); + } finally { + cancelDelegationTokens(jobContext); + } + } + + public static final String SUCCEEDED_FILE_NAME = "_SUCCESS"; + static final String SUCCESSFUL_JOB_OUTPUT_DIR_MARKER = + "mapreduce.fileoutputcommitter.marksuccessfuljobs"; + + private static boolean getOutputDirMarking(Configuration conf) { + return conf.getBoolean(SUCCESSFUL_JOB_OUTPUT_DIR_MARKER, + false); + } + + @Override + public void commitJob(JobContext jobContext) throws IOException { + try { + if (dynamicPartitioningUsed) { + discoverPartitions(jobContext); + // Commit each partition so it gets moved out of the job work + // dir + for (JobContext context : contextDiscoveredByPath.values()) { + new JobConf(context.getConfiguration()) + .getOutputCommitter().commitJob(context); + } + } + if (getBaseOutputCommitter() != null && !dynamicPartitioningUsed) { + getBaseOutputCommitter().commitJob( + HCatMapRedUtil.createJobContext(jobContext)); + } + registerPartitions(jobContext); + // create _SUCCESS FILE if so requested. + OutputJobInfo jobInfo = HCatOutputFormat.getJobInfo(jobContext); + if (getOutputDirMarking(jobContext.getConfiguration())) { + Path outputPath = new Path(jobInfo.getLocation()); + FileSystem fileSys = outputPath.getFileSystem(jobContext + .getConfiguration()); + // create a file in the folder to mark it + if (fileSys.exists(outputPath)) { + Path filePath = new Path(outputPath, + SUCCEEDED_FILE_NAME); + if (!fileSys.exists(filePath)) { // may have been + // created by + // baseCommitter.commitJob() + fileSys.create(filePath).close(); + } + } + } + } finally { + cancelDelegationTokens(jobContext); + } + } + + @Override + public void cleanupJob(JobContext context) throws IOException { + throw new IOException("The method cleanupJob is deprecated and should not be called."); + } + + private String getPartitionRootLocation(String ptnLocn, int numPtnKeys) { + if (ptnRootLocation == null) { + // we only need to calculate it once, it'll be the same for other partitions in this job. + Path ptnRoot = new Path(ptnLocn); + for (int i = 0; i < numPtnKeys; i++) { +// LOG.info("Getting parent of "+ptnRoot.getName()); + ptnRoot = ptnRoot.getParent(); + } + ptnRootLocation = ptnRoot.toString(); + } +// LOG.info("Returning final parent : "+ptnRootLocation); + return ptnRootLocation; + } + + /** + * Generate partition metadata object to be used to add to metadata. + * @param context The job context. + * @param jobInfo The OutputJobInfo. + * @param partLocnRoot The table-equivalent location root of the partition + * (temporary dir if dynamic partition, table dir if static) + * @param partKVs The keyvalue pairs that form the partition + * @param outputSchema The output schema for the partition + * @param params The parameters to store inside the partition + * @param table The Table metadata object under which this Partition will reside + * @param fs FileSystem object to operate on the underlying filesystem + * @param grpName Group name that owns the table dir + * @param perms FsPermission that's the default permission of the table dir. + * @return Constructed Partition metadata object + * @throws java.io.IOException + */ + + private Partition constructPartition( + JobContext context, OutputJobInfo jobInfo, + String partLocnRoot, Map partKVs, + HCatSchema outputSchema, Map params, + Table table, FileSystem fs, + String grpName, FsPermission perms) throws IOException { + + Partition partition = new Partition(); + partition.setDbName(table.getDbName()); + partition.setTableName(table.getTableName()); + partition.setSd(new StorageDescriptor(table.getTTable().getSd())); + + List fields = new ArrayList(); + for (HCatFieldSchema fieldSchema : outputSchema.getFields()) { + fields.add(HCatSchemaUtils.getFieldSchema(fieldSchema)); + } + + partition.getSd().setCols(fields); + + partition.setValues(FileOutputFormatContainer.getPartitionValueList(table, partKVs)); + + partition.setParameters(params); + + // Sets permissions and group name on partition dirs and files. + + Path partPath; + if (Boolean.valueOf((String)table.getProperty("EXTERNAL")) + && jobInfo.getLocation() != null && jobInfo.getLocation().length() > 0) { + // honor external table that specifies the location + partPath = new Path(jobInfo.getLocation()); + } else { + partPath = new Path(partLocnRoot); + int i = 0; + for (FieldSchema partKey : table.getPartitionKeys()) { + if (i++ != 0) { + applyGroupAndPerms(fs, partPath, perms, grpName, false); + } + partPath = constructPartialPartPath(partPath, partKey.getName().toLowerCase(), partKVs); + } + } + + // Apply the group and permissions to the leaf partition and files. + // Need not bother in case of HDFS as permission is taken care of by setting UMask + if (!ShimLoader.getHadoopShims().getHCatShim().isFileInHDFS(fs, partPath)) { + applyGroupAndPerms(fs, partPath, perms, grpName, true); + } + + // Set the location in the StorageDescriptor + if (dynamicPartitioningUsed) { + String dynamicPartitionDestination = getFinalDynamicPartitionDestination(table, partKVs); + if (harProcessor.isEnabled()) { + harProcessor.exec(context, partition, partPath); + partition.getSd().setLocation( + harProcessor.getProcessedLocation(new Path(dynamicPartitionDestination))); + } else { + partition.getSd().setLocation(dynamicPartitionDestination); + } + } else { + partition.getSd().setLocation(partPath.toString()); + } + return partition; + } + + private void applyGroupAndPerms(FileSystem fs, Path dir, FsPermission permission, + String group, boolean recursive) + throws IOException { + fs.setPermission(dir, permission); + if (recursive) { + for (FileStatus fileStatus : fs.listStatus(dir)) { + if (fileStatus.isDir()) { + applyGroupAndPerms(fs, fileStatus.getPath(), permission, group, true); + } else { + fs.setPermission(fileStatus.getPath(), permission); + } + } + } + } + + private String getFinalDynamicPartitionDestination(Table table, Map partKVs) { + // file:///tmp/hcat_junit_warehouse/employee/_DYN0.7770480401313761/emp_country=IN/emp_state=KA -> + // file:///tmp/hcat_junit_warehouse/employee/emp_country=IN/emp_state=KA + Path partPath = new Path(table.getTTable().getSd().getLocation()); + for (FieldSchema partKey : table.getPartitionKeys()) { + partPath = constructPartialPartPath(partPath, partKey.getName().toLowerCase(), partKVs); + } + return partPath.toString(); + } + + private Map getStorerParameterMap(StorerInfo storer) { + Map params = new HashMap(); + + //Copy table level hcat.* keys to the partition + for (Entry entry : storer.getProperties().entrySet()) { + params.put(entry.getKey().toString(), entry.getValue().toString()); + } + return params; + } + + private Path constructPartialPartPath(Path partialPath, String partKey, Map partKVs) { + + StringBuilder sb = new StringBuilder(FileUtils.escapePathName(partKey)); + sb.append("="); + sb.append(FileUtils.escapePathName(partKVs.get(partKey))); + return new Path(partialPath, sb.toString()); + } + + /** + * Update table schema, adding new columns as added for the partition. + * @param client the client + * @param table the table + * @param partitionSchema the schema of the partition + * @throws java.io.IOException Signals that an I/O exception has occurred. + * @throws org.apache.hadoop.hive.metastore.api.InvalidOperationException the invalid operation exception + * @throws org.apache.hadoop.hive.metastore.api.MetaException the meta exception + * @throws org.apache.thrift.TException the t exception + */ + private void updateTableSchema(HiveMetaStoreClient client, Table table, + HCatSchema partitionSchema) throws IOException, InvalidOperationException, MetaException, TException { + + + List newColumns = HCatUtil.validatePartitionSchema(table, partitionSchema); + + if (newColumns.size() != 0) { + List tableColumns = new ArrayList(table.getTTable().getSd().getCols()); + tableColumns.addAll(newColumns); + + //Update table schema to add the newly added columns + table.getTTable().getSd().setCols(tableColumns); + client.alter_table(table.getDbName(), table.getTableName(), table.getTTable()); + } + } + + /** + * Move all of the files from the temp directory to the final location + * @param fs the output file system + * @param file the file to move + * @param srcDir the source directory + * @param destDir the target directory + * @param dryRun - a flag that simply tests if this move would succeed or not based + * on whether other files exist where we're trying to copy + * @throws java.io.IOException + */ + private void moveTaskOutputs(FileSystem fs, + Path file, + Path srcDir, + Path destDir, final boolean dryRun) throws IOException { + + if (file.getName().equals(TEMP_DIR_NAME) || file.getName().equals(LOGS_DIR_NAME) || file.getName().equals(SUCCEEDED_FILE_NAME)) { + return; + } + final Path finalOutputPath = getFinalPath(file, srcDir, destDir); + if (fs.isFile(file)) { + if (dryRun){ + if(LOG.isDebugEnabled()) { + LOG.debug("Testing if moving file: [" + file + "] to [" + + finalOutputPath + "] would cause a problem"); + } + if (fs.exists(finalOutputPath)) { + throw new HCatException(ErrorType.ERROR_MOVE_FAILED, "Data already exists in " + finalOutputPath + + ", duplicate publish not possible."); + } + } else { + if(LOG.isDebugEnabled()) { + LOG.debug("Moving file: [ " + file + "] to [" + finalOutputPath + "]"); + } + // Make sure the parent directory exists. It is not an error + // to recreate an existing directory + fs.mkdirs(finalOutputPath.getParent()); + if (!fs.rename(file, finalOutputPath)) { + if (!fs.delete(finalOutputPath, true)) { + throw new HCatException(ErrorType.ERROR_MOVE_FAILED, "Failed to delete existing path " + finalOutputPath); + } + if (!fs.rename(file, finalOutputPath)) { + throw new HCatException(ErrorType.ERROR_MOVE_FAILED, "Failed to move output to " + finalOutputPath); + } + } + } + } else if(fs.getFileStatus(file).isDir()) { + FileStatus[] children = fs.listStatus(file); + FileStatus firstChild = null; + if (children != null) { + int index=0; + while (index < children.length) { + if (!children[index].getPath().getName().equals(TEMP_DIR_NAME) && !children[index].getPath().getName().equals(LOGS_DIR_NAME) && !children[index].getPath().getName().equals(SUCCEEDED_FILE_NAME)) { + firstChild = children[index]; + break; + } + index++; + } + } + if(firstChild!=null && firstChild.isDir()) { + // If the first child is directory, then rest would be directory too according to HCatalog dir structure + // recurse in that case + for (FileStatus child : children) { + moveTaskOutputs(fs, child.getPath(), srcDir, destDir, dryRun); + } + } else { + + if (!dryRun) { + if (dynamicPartitioningUsed) { + // Optimization: if the first child is file, we have reached the leaf directory, move the parent directory itself + // instead of moving each file under the directory. See HCATALOG-538 + + final Path parentDir = finalOutputPath.getParent(); + // Create the directory + Path placeholder = new Path(parentDir, "_placeholder"); + if (fs.mkdirs(parentDir)) { + // It is weired but we need a placeholder, + // otherwise rename cannot move file to the right place + fs.create(placeholder).close(); + } + if (LOG.isDebugEnabled()) { + LOG.debug("Moving directory: " + file + " to " + parentDir); + } + if (!fs.rename(file, parentDir)) { + final String msg = "Failed to move file: " + file + " to " + parentDir; + LOG.error(msg); + throw new HCatException(ErrorType.ERROR_MOVE_FAILED, msg); + } + fs.delete(placeholder, false); + } else { + // In case of no partition we have to move each file + for (FileStatus child : children) { + moveTaskOutputs(fs, child.getPath(), srcDir, destDir, dryRun); + } + } + } else { + if(fs.exists(finalOutputPath)) { + throw new HCatException(ErrorType.ERROR_MOVE_FAILED, "Data already exists in " + finalOutputPath + + ", duplicate publish not possible."); + } + } + } + } else { + // Should never happen + final String msg = "Unknown file type being asked to be moved, erroring out"; + throw new HCatException(ErrorType.ERROR_MOVE_FAILED, msg); + } + } + + /** + * Find the final name of a given output file, given the output directory + * and the work directory. + * @param file the file to move + * @param src the source directory + * @param dest the target directory + * @return the final path for the specific output file + * @throws java.io.IOException + */ + private Path getFinalPath(Path file, Path src, + Path dest) throws IOException { + URI taskOutputUri = file.toUri(); + URI relativePath = src.toUri().relativize(taskOutputUri); + if (taskOutputUri == relativePath) { + throw new HCatException(ErrorType.ERROR_MOVE_FAILED, "Can not get the relative path: base = " + + src + " child = " + file); + } + if (relativePath.getPath().length() > 0) { + return new Path(dest, relativePath.getPath()); + } else { + return dest; + } + } + + /** + * Run to discover dynamic partitions available + */ + private void discoverPartitions(JobContext context) throws IOException { + if (!partitionsDiscovered) { + // LOG.info("discover ptns called"); + OutputJobInfo jobInfo = HCatOutputFormat.getJobInfo(context); + + harProcessor.setEnabled(jobInfo.getHarRequested()); + + List dynamicPartCols = jobInfo.getPosOfDynPartCols(); + int maxDynamicPartitions = jobInfo.getMaxDynamicPartitions(); + + Path loadPath = new Path(jobInfo.getLocation()); + FileSystem fs = loadPath.getFileSystem(context.getConfiguration()); + + // construct a path pattern (e.g., /*/*) to find all dynamically generated paths + String dynPathSpec = loadPath.toUri().getPath(); + dynPathSpec = dynPathSpec.replaceAll("__HIVE_DEFAULT_PARTITION__", "*"); + + // LOG.info("Searching for "+dynPathSpec); + Path pathPattern = new Path(dynPathSpec); + FileStatus[] status = fs.globStatus(pathPattern); + + partitionsDiscoveredByPath = new LinkedHashMap>(); + contextDiscoveredByPath = new LinkedHashMap(); + + + if (status.length == 0) { + // LOG.warn("No partition found genereated by dynamic partitioning in [" + // +loadPath+"] with depth["+jobInfo.getTable().getPartitionKeysSize() + // +"], dynSpec["+dynPathSpec+"]"); + } else { + if ((maxDynamicPartitions != -1) && (status.length > maxDynamicPartitions)) { + this.partitionsDiscovered = true; + throw new HCatException(ErrorType.ERROR_TOO_MANY_DYNAMIC_PTNS, + "Number of dynamic partitions being created " + + "exceeds configured max allowable partitions[" + + maxDynamicPartitions + + "], increase parameter [" + + HiveConf.ConfVars.DYNAMICPARTITIONMAXPARTS.varname + + "] if needed."); + } + + for (FileStatus st : status) { + LinkedHashMap fullPartSpec = new LinkedHashMap(); + Warehouse.makeSpecFromName(fullPartSpec, st.getPath()); + partitionsDiscoveredByPath.put(st.getPath().toString(), fullPartSpec); + JobConf jobConf = (JobConf)context.getConfiguration(); + JobContext currContext = HCatMapRedUtil.createJobContext( + jobConf, + context.getJobID(), + InternalUtil.createReporter(HCatMapRedUtil.createTaskAttemptContext(jobConf, + ShimLoader.getHadoopShims().getHCatShim().createTaskAttemptID()))); + HCatOutputFormat.configureOutputStorageHandler(currContext, jobInfo, fullPartSpec); + contextDiscoveredByPath.put(st.getPath().toString(), currContext); + } + } + + // for (Entry> spec : partitionsDiscoveredByPath.entrySet()){ + // LOG.info("Partition "+ spec.getKey()); + // for (Entry e : spec.getValue().entrySet()){ + // LOG.info(e.getKey() + "=>" +e.getValue()); + // } + // } + + this.partitionsDiscovered = true; + } + } + + private void registerPartitions(JobContext context) throws IOException{ + if (dynamicPartitioningUsed){ + discoverPartitions(context); + } + OutputJobInfo jobInfo = HCatOutputFormat.getJobInfo(context); + Configuration conf = context.getConfiguration(); + Table table = new Table(jobInfo.getTableInfo().getTable()); + Path tblPath = new Path(table.getTTable().getSd().getLocation()); + FileSystem fs = tblPath.getFileSystem(conf); + + if( table.getPartitionKeys().size() == 0 ) { + //Move data from temp directory the actual table directory + //No metastore operation required. + Path src = new Path(jobInfo.getLocation()); + moveTaskOutputs(fs, src, src, tblPath, false); + fs.delete(src, true); + return; + } + + HiveMetaStoreClient client = null; + HCatTableInfo tableInfo = jobInfo.getTableInfo(); + List partitionsAdded = new ArrayList(); + try { + HiveConf hiveConf = HCatUtil.getHiveConf(conf); + client = HCatUtil.getHiveClient(hiveConf); + StorerInfo storer = InternalUtil.extractStorerInfo(table.getTTable().getSd(),table.getParameters()); + + FileStatus tblStat = fs.getFileStatus(tblPath); + String grpName = tblStat.getGroup(); + FsPermission perms = tblStat.getPermission(); + + List partitionsToAdd = new ArrayList(); + if (!dynamicPartitioningUsed){ + partitionsToAdd.add( + constructPartition( + context,jobInfo, + tblPath.toString(), jobInfo.getPartitionValues() + ,jobInfo.getOutputSchema(), getStorerParameterMap(storer) + ,table, fs + ,grpName,perms)); + }else{ + for (Entry> entry : partitionsDiscoveredByPath.entrySet()){ + partitionsToAdd.add( + constructPartition( + context,jobInfo, + getPartitionRootLocation(entry.getKey(),entry.getValue().size()), entry.getValue() + ,jobInfo.getOutputSchema(), getStorerParameterMap(storer) + ,table, fs + ,grpName,perms)); + } + } + + ArrayList> ptnInfos = new ArrayList>(); + for(Partition ptn : partitionsToAdd){ + ptnInfos.add(InternalUtil.createPtnKeyValueMap(new Table(tableInfo.getTable()), ptn)); + } + + //Publish the new partition(s) + if (dynamicPartitioningUsed && harProcessor.isEnabled() && (!partitionsToAdd.isEmpty())){ + + Path src = new Path(ptnRootLocation); + // check here for each dir we're copying out, to see if it + // already exists, error out if so + moveTaskOutputs(fs, src, src, tblPath, true); + moveTaskOutputs(fs, src, src, tblPath, false); + fs.delete(src, true); + try { + updateTableSchema(client, table, jobInfo.getOutputSchema()); + LOG.info("HAR is being used. The table {} has new partitions {}.", table.getTableName(), ptnInfos); + client.add_partitions(partitionsToAdd); + partitionsAdded = partitionsToAdd; + } catch (Exception e){ + // There was an error adding partitions : rollback fs copy and rethrow + for (Partition p : partitionsToAdd){ + Path ptnPath = new Path(harProcessor.getParentFSPath(new Path(p.getSd().getLocation()))); + if (fs.exists(ptnPath)){ + fs.delete(ptnPath,true); + } + } + throw e; + } + + }else{ + // no harProcessor, regular operation + updateTableSchema(client, table, jobInfo.getOutputSchema()); + LOG.info("HAR not is not being used. The table {} has new partitions {}.", table.getTableName(), ptnInfos); + if (dynamicPartitioningUsed && (partitionsToAdd.size()>0)){ + Path src = new Path(ptnRootLocation); + moveTaskOutputs(fs, src, src, tblPath, true); + moveTaskOutputs(fs, src, src, tblPath, false); + fs.delete(src, true); + } + client.add_partitions(partitionsToAdd); + partitionsAdded = partitionsToAdd; + } + } catch (Exception e) { + if (partitionsAdded.size() > 0) { + try { + // baseCommitter.cleanupJob failed, try to clean up the + // metastore + for (Partition p : partitionsAdded) { + client.dropPartition(tableInfo.getDatabaseName(), + tableInfo.getTableName(), p.getValues()); + } + } catch (Exception te) { + // Keep cause as the original exception + throw new HCatException( + ErrorType.ERROR_PUBLISHING_PARTITION, e); + } + } + if (e instanceof HCatException) { + throw (HCatException) e; + } else { + throw new HCatException(ErrorType.ERROR_PUBLISHING_PARTITION, e); + } + } finally { + HCatUtil.closeHiveClientQuietly(client); + } + } + + private void cancelDelegationTokens(JobContext context) throws IOException{ + LOG.info("Cancelling deletgation token for the job."); + HiveMetaStoreClient client = null; + try { + HiveConf hiveConf = HCatUtil + .getHiveConf(context.getConfiguration()); + client = HCatUtil.getHiveClient(hiveConf); + // cancel the deleg. tokens that were acquired for this job now that + // we are done - we should cancel if the tokens were acquired by + // HCatOutputFormat and not if they were supplied by Oozie. + // In the latter case the HCAT_KEY_TOKEN_SIGNATURE property in + // the conf will not be set + String tokenStrForm = client.getTokenStrForm(); + if (tokenStrForm != null + && context.getConfiguration().get( + HCatConstants.HCAT_KEY_TOKEN_SIGNATURE) != null) { + client.cancelDelegationToken(tokenStrForm); + } + } catch (MetaException e) { + LOG.warn("MetaException while cancelling delegation token.", e); + } catch (TException e) { + LOG.warn("TException while cancelling delegation token.", e); + } finally { + HCatUtil.closeHiveClientQuietly(client); + } + } + + +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/FileOutputFormatContainer.java hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/FileOutputFormatContainer.java new file mode 100644 index 0000000..bd1d98d --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/FileOutputFormatContainer.java @@ -0,0 +1,253 @@ +/** + * 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.hcatalog.mapreduce; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.PathFilter; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.serde2.SerDe; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.io.WritableComparable; +import org.apache.hadoop.mapred.FileOutputFormat; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.OutputCommitter; +import org.apache.hadoop.mapreduce.RecordWriter; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter; +import org.apache.hadoop.util.ReflectionUtils; +import org.apache.hcatalog.common.ErrorType; +import org.apache.hcatalog.common.HCatConstants; +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.common.HCatUtil; +import org.apache.hcatalog.data.HCatRecord; +import org.apache.thrift.TException; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * File-based storage (ie RCFile, Text, etc) implementation of OutputFormatContainer. + * This implementation supports the following HCatalog features: partitioning, dynamic partitioning, Hadoop Archiving, etc. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.FileOutputFormatContainer} instead + */ +class FileOutputFormatContainer extends OutputFormatContainer { + + private static final PathFilter hiddenFileFilter = new PathFilter() { + public boolean accept(Path p) { + String name = p.getName(); + return !name.startsWith("_") && !name.startsWith("."); + } + }; + + /** + * @param of base OutputFormat to contain + */ + public FileOutputFormatContainer(org.apache.hadoop.mapred.OutputFormat, ? super Writable> of) { + super(of); + } + + @Override + public RecordWriter, HCatRecord> getRecordWriter(TaskAttemptContext context) throws IOException, InterruptedException { + //this needs to be manually set, under normal circumstances MR Task does this + setWorkOutputPath(context); + + //Configure the output key and value classes. + // This is required for writing null as key for file based tables. + context.getConfiguration().set("mapred.output.key.class", + NullWritable.class.getName()); + String jobInfoString = context.getConfiguration().get( + HCatConstants.HCAT_KEY_OUTPUT_INFO); + OutputJobInfo jobInfo = (OutputJobInfo) HCatUtil + .deserialize(jobInfoString); + StorerInfo storeInfo = jobInfo.getTableInfo().getStorerInfo(); + HCatStorageHandler storageHandler = HCatUtil.getStorageHandler( + context.getConfiguration(), storeInfo); + Class serde = storageHandler.getSerDeClass(); + SerDe sd = (SerDe) ReflectionUtils.newInstance(serde, + context.getConfiguration()); + context.getConfiguration().set("mapred.output.value.class", + sd.getSerializedClass().getName()); + + RecordWriter, HCatRecord> rw; + if (HCatBaseOutputFormat.getJobInfo(context).isDynamicPartitioningUsed()){ + // When Dynamic partitioning is used, the RecordWriter instance initialized here isn't used. Can use null. + // (That's because records can't be written until the values of the dynamic partitions are deduced. + // By that time, a new local instance of RecordWriter, with the correct output-path, will be constructed.) + rw = new FileRecordWriterContainer((org.apache.hadoop.mapred.RecordWriter)null,context); + } else { + Path parentDir = new Path(context.getConfiguration().get("mapred.work.output.dir")); + Path childPath = new Path(parentDir,FileOutputFormat.getUniqueName(new JobConf(context.getConfiguration()), "part")); + + rw = new FileRecordWriterContainer( + getBaseOutputFormat().getRecordWriter( + parentDir.getFileSystem(context.getConfiguration()), + new JobConf(context.getConfiguration()), + childPath.toString(), + InternalUtil.createReporter(context)), + context); + } + return rw; + } + + @Override + public void checkOutputSpecs(JobContext context) throws IOException, InterruptedException { + OutputJobInfo jobInfo = HCatOutputFormat.getJobInfo(context); + HiveMetaStoreClient client = null; + try { + HiveConf hiveConf = HCatUtil.getHiveConf(context.getConfiguration()); + client = HCatUtil.getHiveClient(hiveConf); + handleDuplicatePublish(context, + jobInfo, + client, + new Table(jobInfo.getTableInfo().getTable())); + } catch (MetaException e) { + throw new IOException(e); + } catch (TException e) { + throw new IOException(e); + } finally { + HCatUtil.closeHiveClientQuietly(client); + } + + if (!jobInfo.isDynamicPartitioningUsed()) { + JobConf jobConf = new JobConf(context.getConfiguration()); + getBaseOutputFormat().checkOutputSpecs(null, jobConf); + //checkoutputspecs might've set some properties we need to have context reflect that + HCatUtil.copyConf(jobConf, context.getConfiguration()); + } + } + + @Override + public OutputCommitter getOutputCommitter(TaskAttemptContext context) throws IOException, InterruptedException { + //this needs to be manually set, under normal circumstances MR Task does this + setWorkOutputPath(context); + return new FileOutputCommitterContainer(context, + HCatBaseOutputFormat.getJobInfo(context).isDynamicPartitioningUsed() ? + null : + new JobConf(context.getConfiguration()).getOutputCommitter()); + } + + /** + * Handles duplicate publish of partition. Fails if partition already exists. + * For non partitioned tables, fails if files are present in table directory. + * For dynamic partitioned publish, does nothing - check would need to be done at recordwriter time + * @param context the job + * @param outputInfo the output info + * @param client the metastore client + * @param table the table being written to + * @throws IOException + * @throws org.apache.hadoop.hive.metastore.api.MetaException + * @throws org.apache.thrift.TException + */ + private static void handleDuplicatePublish(JobContext context, OutputJobInfo outputInfo, + HiveMetaStoreClient client, Table table) throws IOException, MetaException, TException, NoSuchObjectException { + + /* + * For fully specified ptn, follow strict checks for existence of partitions in metadata + * For unpartitioned tables, follow filechecks + * For partially specified tables: + * This would then need filechecks at the start of a ptn write, + * Doing metadata checks can get potentially very expensive (fat conf) if + * there are a large number of partitions that match the partial specifications + */ + + if (table.getPartitionKeys().size() > 0) { + if (!outputInfo.isDynamicPartitioningUsed()) { + List partitionValues = getPartitionValueList( + table, outputInfo.getPartitionValues()); + // fully-specified partition + List currentParts = client.listPartitionNames(outputInfo.getDatabaseName(), + outputInfo.getTableName(), partitionValues, (short) 1); + + if (currentParts.size() > 0) { + throw new HCatException(ErrorType.ERROR_DUPLICATE_PARTITION); + } + } + } else { + List partitionValues = getPartitionValueList( + table, outputInfo.getPartitionValues()); + // non-partitioned table + + Path tablePath = new Path(table.getTTable().getSd().getLocation()); + FileSystem fs = tablePath.getFileSystem(context.getConfiguration()); + + if (fs.exists(tablePath)) { + FileStatus[] status = fs.globStatus(new Path(tablePath, "*"), hiddenFileFilter); + + if (status.length > 0) { + throw new HCatException(ErrorType.ERROR_NON_EMPTY_TABLE, + table.getDbName() + "." + table.getTableName()); + } + } + } + } + + /** + * Convert the partition value map to a value list in the partition key order. + * @param table the table being written to + * @param valueMap the partition value map + * @return the partition value list + * @throws java.io.IOException + */ + static List getPartitionValueList(Table table, Map valueMap) throws IOException { + + if (valueMap.size() != table.getPartitionKeys().size()) { + throw new HCatException(ErrorType.ERROR_INVALID_PARTITION_VALUES, + "Table " + + table.getTableName() + " has " + + table.getPartitionKeys().size() + " partition keys, got " + + valueMap.size()); + } + + List values = new ArrayList(); + + for (FieldSchema schema : table.getPartitionKeys()) { + String value = valueMap.get(schema.getName().toLowerCase()); + + if (value == null) { + throw new HCatException(ErrorType.ERROR_MISSING_PARTITION_KEY, + "Key " + schema.getName() + " of table " + table.getTableName()); + } + + values.add(value); + } + + return values; + } + + static void setWorkOutputPath(TaskAttemptContext context) throws IOException { + String outputPath = context.getConfiguration().get("mapred.output.dir"); + //we need to do this to get the task path and set it for mapred implementation + //since it can't be done automatically because of mapreduce->mapred abstraction + if (outputPath != null) + context.getConfiguration().set("mapred.work.output.dir", + new FileOutputCommitter(new Path(outputPath), context).getWorkPath().toString()); + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/FileRecordWriterContainer.java hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/FileRecordWriterContainer.java new file mode 100644 index 0000000..b781f16 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/FileRecordWriterContainer.java @@ -0,0 +1,268 @@ +/** + * 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.hcatalog.mapreduce; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.serde2.SerDe; +import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.io.WritableComparable; +import org.apache.hive.hcatalog.mapreduce.HCatMapRedUtil; +import org.apache.hadoop.mapred.Reporter; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.OutputCommitter; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter; +import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; +import org.apache.hadoop.util.ReflectionUtils; +import org.apache.hcatalog.common.ErrorType; +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.common.HCatUtil; +import org.apache.hcatalog.data.HCatRecord; + +/** + * Part of the FileOutput*Container classes + * See {@link FileOutputFormatContainer} for more information + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.FileRecordWriterContainer} instead + */ +class FileRecordWriterContainer extends RecordWriterContainer { + + private final HCatStorageHandler storageHandler; + private final SerDe serDe; + private final ObjectInspector objectInspector; + + private boolean dynamicPartitioningUsed = false; + + private final Map, ? super Writable>> baseDynamicWriters; + private final Map baseDynamicSerDe; + private final Map baseDynamicCommitters; + private final Map dynamicContexts; + private final Map dynamicObjectInspectors; + private Map dynamicOutputJobInfo; + + + private final List partColsToDel; + private final List dynamicPartCols; + private int maxDynamicPartitions; + + private OutputJobInfo jobInfo; + private TaskAttemptContext context; + + /** + * @param baseWriter RecordWriter to contain + * @param context current TaskAttemptContext + * @throws IOException + * @throws InterruptedException + */ + public FileRecordWriterContainer(org.apache.hadoop.mapred.RecordWriter, ? super Writable> baseWriter, + TaskAttemptContext context) throws IOException, InterruptedException { + super(context, baseWriter); + this.context = context; + jobInfo = HCatOutputFormat.getJobInfo(context); + + storageHandler = HCatUtil.getStorageHandler(context.getConfiguration(), jobInfo.getTableInfo().getStorerInfo()); + serDe = ReflectionUtils.newInstance(storageHandler.getSerDeClass(), context.getConfiguration()); + objectInspector = InternalUtil.createStructObjectInspector(jobInfo.getOutputSchema()); + try { + InternalUtil.initializeOutputSerDe(serDe, context.getConfiguration(), jobInfo); + } catch (SerDeException e) { + throw new IOException("Failed to inialize SerDe", e); + } + + // If partition columns occur in data, we want to remove them. + partColsToDel = jobInfo.getPosOfPartCols(); + dynamicPartitioningUsed = jobInfo.isDynamicPartitioningUsed(); + dynamicPartCols = jobInfo.getPosOfDynPartCols(); + maxDynamicPartitions = jobInfo.getMaxDynamicPartitions(); + + if ((partColsToDel == null) || (dynamicPartitioningUsed && (dynamicPartCols == null))) { + throw new HCatException("It seems that setSchema() is not called on " + + "HCatOutputFormat. Please make sure that method is called."); + } + + + if (!dynamicPartitioningUsed) { + this.baseDynamicSerDe = null; + this.baseDynamicWriters = null; + this.baseDynamicCommitters = null; + this.dynamicContexts = null; + this.dynamicObjectInspectors = null; + this.dynamicOutputJobInfo = null; + } else { + this.baseDynamicSerDe = new HashMap(); + this.baseDynamicWriters = new HashMap, ? super Writable>>(); + this.baseDynamicCommitters = new HashMap(); + this.dynamicContexts = new HashMap(); + this.dynamicObjectInspectors = new HashMap(); + this.dynamicOutputJobInfo = new HashMap(); + } + } + + /** + * @return the storagehandler + */ + public HCatStorageHandler getStorageHandler() { + return storageHandler; + } + + @Override + public void close(TaskAttemptContext context) throws IOException, + InterruptedException { + Reporter reporter = InternalUtil.createReporter(context); + if (dynamicPartitioningUsed) { + for (org.apache.hadoop.mapred.RecordWriter, ? super Writable> bwriter : baseDynamicWriters.values()) { + //We are in RecordWriter.close() make sense that the context would be TaskInputOutput + bwriter.close(reporter); + } + for (Map.Entry entry : baseDynamicCommitters.entrySet()) { + org.apache.hadoop.mapred.TaskAttemptContext currContext = dynamicContexts.get(entry.getKey()); + OutputCommitter baseOutputCommitter = entry.getValue(); + if (baseOutputCommitter.needsTaskCommit(currContext)) { + baseOutputCommitter.commitTask(currContext); + } + } + } else { + getBaseRecordWriter().close(reporter); + } + } + + @Override + public void write(WritableComparable key, HCatRecord value) throws IOException, + InterruptedException { + + org.apache.hadoop.mapred.RecordWriter localWriter; + ObjectInspector localObjectInspector; + SerDe localSerDe; + OutputJobInfo localJobInfo = null; + + if (dynamicPartitioningUsed) { + // calculate which writer to use from the remaining values - this needs to be done before we delete cols + List dynamicPartValues = new ArrayList(); + for (Integer colToAppend : dynamicPartCols) { + dynamicPartValues.add(value.get(colToAppend).toString()); + } + + String dynKey = dynamicPartValues.toString(); + if (!baseDynamicWriters.containsKey(dynKey)) { + if ((maxDynamicPartitions != -1) && (baseDynamicWriters.size() > maxDynamicPartitions)) { + throw new HCatException(ErrorType.ERROR_TOO_MANY_DYNAMIC_PTNS, + "Number of dynamic partitions being created " + + "exceeds configured max allowable partitions[" + + maxDynamicPartitions + + "], increase parameter [" + + HiveConf.ConfVars.DYNAMICPARTITIONMAXPARTS.varname + + "] if needed."); + } + + org.apache.hadoop.mapred.TaskAttemptContext currTaskContext = HCatMapRedUtil.createTaskAttemptContext(context); + configureDynamicStorageHandler(currTaskContext, dynamicPartValues); + localJobInfo = HCatBaseOutputFormat.getJobInfo(currTaskContext); + + //setup serDe + SerDe currSerDe = ReflectionUtils.newInstance(storageHandler.getSerDeClass(), currTaskContext.getJobConf()); + try { + InternalUtil.initializeOutputSerDe(currSerDe, currTaskContext.getConfiguration(), localJobInfo); + } catch (SerDeException e) { + throw new IOException("Failed to initialize SerDe", e); + } + + //create base OutputFormat + org.apache.hadoop.mapred.OutputFormat baseOF = + ReflectionUtils.newInstance(storageHandler.getOutputFormatClass(), currTaskContext.getJobConf()); + + //We are skipping calling checkOutputSpecs() for each partition + //As it can throw a FileAlreadyExistsException when more than one mapper is writing to a partition + //See HCATALOG-490, also to avoid contacting the namenode for each new FileOutputFormat instance + //In general this should be ok for most FileOutputFormat implementations + //but may become an issue for cases when the method is used to perform other setup tasks + + //get Output Committer + org.apache.hadoop.mapred.OutputCommitter baseOutputCommitter = currTaskContext.getJobConf().getOutputCommitter(); + //create currJobContext the latest so it gets all the config changes + org.apache.hadoop.mapred.JobContext currJobContext = HCatMapRedUtil.createJobContext(currTaskContext); + //setupJob() + baseOutputCommitter.setupJob(currJobContext); + //recreate to refresh jobConf of currTask context + currTaskContext = + HCatMapRedUtil.createTaskAttemptContext(currJobContext.getJobConf(), + currTaskContext.getTaskAttemptID(), + currTaskContext.getProgressible()); + //set temp location + currTaskContext.getConfiguration().set("mapred.work.output.dir", + new FileOutputCommitter(new Path(localJobInfo.getLocation()), currTaskContext).getWorkPath().toString()); + //setupTask() + baseOutputCommitter.setupTask(currTaskContext); + + Path parentDir = new Path(currTaskContext.getConfiguration().get("mapred.work.output.dir")); + Path childPath = new Path(parentDir,FileOutputFormat.getUniqueFile(currTaskContext, "part", "")); + + org.apache.hadoop.mapred.RecordWriter baseRecordWriter = + baseOF.getRecordWriter( + parentDir.getFileSystem(currTaskContext.getConfiguration()), + currTaskContext.getJobConf(), + childPath.toString(), + InternalUtil.createReporter(currTaskContext)); + + baseDynamicWriters.put(dynKey, baseRecordWriter); + baseDynamicSerDe.put(dynKey, currSerDe); + baseDynamicCommitters.put(dynKey, baseOutputCommitter); + dynamicContexts.put(dynKey, currTaskContext); + dynamicObjectInspectors.put(dynKey, InternalUtil.createStructObjectInspector(jobInfo.getOutputSchema())); + dynamicOutputJobInfo.put(dynKey, HCatOutputFormat.getJobInfo(dynamicContexts.get(dynKey))); + } + + localJobInfo = dynamicOutputJobInfo.get(dynKey); + localWriter = baseDynamicWriters.get(dynKey); + localSerDe = baseDynamicSerDe.get(dynKey); + localObjectInspector = dynamicObjectInspectors.get(dynKey); + } else { + localJobInfo = jobInfo; + localWriter = getBaseRecordWriter(); + localSerDe = serDe; + localObjectInspector = objectInspector; + } + + for (Integer colToDel : partColsToDel) { + value.remove(colToDel); + } + + + //The key given by user is ignored + try { + localWriter.write(NullWritable.get(), localSerDe.serialize(value.getAll(), localObjectInspector)); + } catch (SerDeException e) { + throw new IOException("Failed to serialize object", e); + } + } + + protected void configureDynamicStorageHandler(JobContext context, List dynamicPartVals) throws IOException { + HCatOutputFormat.configureOutputStorageHandler(context, dynamicPartVals); + } + +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/FosterStorageHandler.java hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/FosterStorageHandler.java new file mode 100644 index 0000000..46ffc9b --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/FosterStorageHandler.java @@ -0,0 +1,194 @@ +/** + * 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.hcatalog.mapreduce; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.common.FileUtils; +import org.apache.hadoop.hive.metastore.HiveMetaHook; +import org.apache.hadoop.hive.ql.io.RCFile; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.plan.TableDesc; +import org.apache.hadoop.hive.ql.security.authorization.DefaultHiveAuthorizationProvider; +import org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider; +import org.apache.hadoop.hive.serde2.SerDe; +import org.apache.hadoop.mapred.InputFormat; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.OutputFormat; +import org.apache.hcatalog.common.HCatConstants; +import org.apache.hcatalog.common.HCatUtil; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +/** + * This class is used to encapsulate the InputFormat, OutputFormat and SerDe + * artifacts of tables which don't define a SerDe. This StorageHandler assumes + * the supplied storage artifacts are for a file-based storage system. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.FosterStorageHandler} instead + */ +public class FosterStorageHandler extends HCatStorageHandler { + + public Configuration conf; + /** The directory under which data is initially written for a partitioned table */ + protected static final String DYNTEMP_DIR_NAME = "_DYN"; + + /** The directory under which data is initially written for a non partitioned table */ + protected static final String TEMP_DIR_NAME = "_TEMP"; + + private Class ifClass; + private Class ofClass; + private Class serDeClass; + + public FosterStorageHandler(String ifName, String ofName, String serdeName) throws ClassNotFoundException { + this((Class) Class.forName(ifName), + (Class) Class.forName(ofName), + (Class) Class.forName(serdeName)); + } + + public FosterStorageHandler(Class ifClass, + Class ofClass, + Class serDeClass) { + this.ifClass = ifClass; + this.ofClass = ofClass; + this.serDeClass = serDeClass; + } + + @Override + public Class getInputFormatClass() { + return ifClass; //To change body of overridden methods use File | Settings | File Templates. + } + + @Override + public Class getOutputFormatClass() { + return ofClass; //To change body of overridden methods use File | Settings | File Templates. + } + + @Override + public Class getSerDeClass() { + return serDeClass; //To change body of implemented methods use File | Settings | File Templates. + } + + @Override + public HiveMetaHook getMetaHook() { + return null; + } + + @Override + public void configureInputJobProperties(TableDesc tableDesc, + Map jobProperties) { + + } + + @Override + public void configureOutputJobProperties(TableDesc tableDesc, + Map jobProperties) { + try { + OutputJobInfo jobInfo = (OutputJobInfo) + HCatUtil.deserialize(tableDesc.getJobProperties().get( + HCatConstants.HCAT_KEY_OUTPUT_INFO)); + String parentPath = jobInfo.getTableInfo().getTableLocation(); + String dynHash = tableDesc.getJobProperties().get( + HCatConstants.HCAT_DYNAMIC_PTN_JOBID); + + // For dynamic partitioned writes without all keyvalues specified, + // we create a temp dir for the associated write job + if (dynHash != null) { + parentPath = new Path(parentPath, + DYNTEMP_DIR_NAME + dynHash).toString(); + } + + String outputLocation; + + if (Boolean.valueOf((String)tableDesc.getProperties().get("EXTERNAL")) + && jobInfo.getLocation() != null && jobInfo.getLocation().length() > 0) { + // honor external table that specifies the location + outputLocation = jobInfo.getLocation(); + } else if (dynHash == null && jobInfo.getPartitionValues().size() == 0) { + // For non-partitioned tables, we send them to the temp dir + outputLocation = TEMP_DIR_NAME; + } else { + List cols = new ArrayList(); + List values = new ArrayList(); + + //Get the output location in the order partition keys are defined for the table. + for (String name : + jobInfo.getTableInfo(). + getPartitionColumns().getFieldNames()) { + String value = jobInfo.getPartitionValues().get(name); + cols.add(name); + values.add(value); + } + outputLocation = FileUtils.makePartName(cols, values); + } + + jobInfo.setLocation(new Path(parentPath, outputLocation).toString()); + + //only set output dir if partition is fully materialized + if (jobInfo.getPartitionValues().size() + == jobInfo.getTableInfo().getPartitionColumns().size()) { + jobProperties.put("mapred.output.dir", jobInfo.getLocation()); + } + + //TODO find a better home for this, RCFile specifc + jobProperties.put(RCFile.COLUMN_NUMBER_CONF_STR, + Integer.toOctalString( + jobInfo.getOutputSchema().getFields().size())); + jobProperties.put(HCatConstants.HCAT_KEY_OUTPUT_INFO, + HCatUtil.serialize(jobInfo)); + } catch (IOException e) { + throw new IllegalStateException("Failed to set output path", e); + } + + } + + @Override + OutputFormatContainer getOutputFormatContainer( + org.apache.hadoop.mapred.OutputFormat outputFormat) { + return new FileOutputFormatContainer(outputFormat); + } + + @Override + public Configuration getConf() { + return conf; + } + + @Override + public void setConf(Configuration conf) { + this.conf = conf; + } + + @Override + public HiveAuthorizationProvider getAuthorizationProvider() + throws HiveException { + return new DefaultHiveAuthorizationProvider(); + } + @Override + public void configureJobConf(TableDesc tableDesc, JobConf jobConf) { + //do nothing by default + //EK: added the same (no-op) implementation as in + // org.apache.hive.hcatalog.DefaultStorageHandler (hive 0.12) + // this is needed to get 0.11 API compat layer to work + // see HIVE-4896 + } + +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/HCatBaseInputFormat.java hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/HCatBaseInputFormat.java new file mode 100644 index 0000000..efb47b9 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/HCatBaseInputFormat.java @@ -0,0 +1,336 @@ +/** + * 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.hcatalog.mapreduce; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.LinkedList; +import java.util.Map; +import java.util.HashMap; +import java.util.List; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.WritableComparable; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapreduce.InputFormat; +import org.apache.hadoop.mapreduce.InputSplit; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.RecordReader; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.util.ReflectionUtils; +import org.apache.hadoop.util.StringUtils; + +import org.apache.hcatalog.common.HCatConstants; +import org.apache.hcatalog.common.HCatUtil; +import org.apache.hcatalog.data.HCatRecord; +import org.apache.hcatalog.data.schema.HCatFieldSchema; +import org.apache.hcatalog.data.schema.HCatSchema; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.HCatBaseInputFormat} instead + */ +public abstract class HCatBaseInputFormat + extends InputFormat { + + /** + * get the schema for the HCatRecord data returned by HCatInputFormat. + * + * @param context the jobContext + * @throws IllegalArgumentException + */ + private Class inputFileFormatClass; + + // TODO needs to go in InitializeInput? as part of InputJobInfo + private static HCatSchema getOutputSchema(Configuration conf) + throws IOException { + String os = conf.get(HCatConstants.HCAT_KEY_OUTPUT_SCHEMA); + if (os == null) { + return getTableSchema(conf); + } else { + return (HCatSchema) HCatUtil.deserialize(os); + } + } + + /** + * Set the schema for the HCatRecord data returned by HCatInputFormat. + * @param job the job object + * @param hcatSchema the schema to use as the consolidated schema + */ + public static void setOutputSchema(Job job, HCatSchema hcatSchema) + throws IOException { + job.getConfiguration().set(HCatConstants.HCAT_KEY_OUTPUT_SCHEMA, + HCatUtil.serialize(hcatSchema)); + } + + protected static org.apache.hadoop.mapred.InputFormat + getMapRedInputFormat(JobConf job, Class inputFormatClass) throws IOException { + return ( + org.apache.hadoop.mapred.InputFormat) + ReflectionUtils.newInstance(inputFormatClass, job); + } + + /** + * Logically split the set of input files for the job. Returns the + * underlying InputFormat's splits + * @param jobContext the job context object + * @return the splits, an HCatInputSplit wrapper over the storage + * handler InputSplits + * @throws IOException or InterruptedException + */ + @Override + public List getSplits(JobContext jobContext) + throws IOException, InterruptedException { + Configuration conf = jobContext.getConfiguration(); + + //Get the job info from the configuration, + //throws exception if not initialized + InputJobInfo inputJobInfo; + try { + inputJobInfo = getJobInfo(conf); + } catch (Exception e) { + throw new IOException(e); + } + + List splits = new ArrayList(); + List partitionInfoList = inputJobInfo.getPartitions(); + if (partitionInfoList == null) { + //No partitions match the specified partition filter + return splits; + } + + HCatStorageHandler storageHandler; + JobConf jobConf; + //For each matching partition, call getSplits on the underlying InputFormat + for (PartInfo partitionInfo : partitionInfoList) { + jobConf = HCatUtil.getJobConfFromContext(jobContext); + setInputPath(jobConf, partitionInfo.getLocation()); + Map jobProperties = partitionInfo.getJobProperties(); + + HCatSchema allCols = new HCatSchema(new LinkedList()); + for (HCatFieldSchema field : + inputJobInfo.getTableInfo().getDataColumns().getFields()) + allCols.append(field); + for (HCatFieldSchema field : + inputJobInfo.getTableInfo().getPartitionColumns().getFields()) + allCols.append(field); + + HCatUtil.copyJobPropertiesToJobConf(jobProperties, jobConf); + + storageHandler = HCatUtil.getStorageHandler( + jobConf, partitionInfo); + + //Get the input format + Class inputFormatClass = storageHandler.getInputFormatClass(); + org.apache.hadoop.mapred.InputFormat inputFormat = + getMapRedInputFormat(jobConf, inputFormatClass); + + //Call getSplit on the InputFormat, create an HCatSplit for each + //underlying split. When the desired number of input splits is missing, + //use a default number (denoted by zero). + //TODO(malewicz): Currently each partition is split independently into + //a desired number. However, we want the union of all partitions to be + //split into a desired number while maintaining balanced sizes of input + //splits. + int desiredNumSplits = + conf.getInt(HCatConstants.HCAT_DESIRED_PARTITION_NUM_SPLITS, 0); + org.apache.hadoop.mapred.InputSplit[] baseSplits = + inputFormat.getSplits(jobConf, desiredNumSplits); + + for (org.apache.hadoop.mapred.InputSplit split : baseSplits) { + splits.add(new HCatSplit( + partitionInfo, + split, allCols)); + } + } + + return splits; + } + + /** + * Create the RecordReader for the given InputSplit. Returns the underlying + * RecordReader if the required operations are supported and schema matches + * with HCatTable schema. Returns an HCatRecordReader if operations need to + * be implemented in HCat. + * @param split the split + * @param taskContext the task attempt context + * @return the record reader instance, either an HCatRecordReader(later) or + * the underlying storage handler's RecordReader + * @throws IOException or InterruptedException + */ + @Override + public RecordReader + createRecordReader(InputSplit split, + TaskAttemptContext taskContext) throws IOException, InterruptedException { + + HCatSplit hcatSplit = InternalUtil.castToHCatSplit(split); + PartInfo partitionInfo = hcatSplit.getPartitionInfo(); + JobContext jobContext = taskContext; + Configuration conf = jobContext.getConfiguration(); + + HCatStorageHandler storageHandler = HCatUtil.getStorageHandler( + conf, partitionInfo); + + JobConf jobConf = HCatUtil.getJobConfFromContext(jobContext); + Map jobProperties = partitionInfo.getJobProperties(); + HCatUtil.copyJobPropertiesToJobConf(jobProperties, jobConf); + + Map valuesNotInDataCols = getColValsNotInDataColumns( + getOutputSchema(conf), partitionInfo + ); + + return new HCatRecordReader(storageHandler, valuesNotInDataCols); + } + + + /** + * gets values for fields requested by output schema which will not be in the data + */ + private static Map getColValsNotInDataColumns(HCatSchema outputSchema, + PartInfo partInfo) { + HCatSchema dataSchema = partInfo.getPartitionSchema(); + Map vals = new HashMap(); + for (String fieldName : outputSchema.getFieldNames()) { + if (dataSchema.getPosition(fieldName) == null) { + // this entry of output is not present in the output schema + // so, we first check the table schema to see if it is a part col + + if (partInfo.getPartitionValues().containsKey(fieldName)) { + vals.put(fieldName, partInfo.getPartitionValues().get(fieldName)); + } else { + vals.put(fieldName, null); + } + } + } + return vals; + } + + /** + * @see org.apache.hcatalog.mapreduce.HCatBaseInputFormat#getTableSchema(org.apache.hadoop.conf.Configuration) + * @deprecated Use {@link #getTableSchema(org.apache.hadoop.conf.Configuration)} + */ + public static HCatSchema getTableSchema(JobContext context) + throws IOException { + return getTableSchema(context.getConfiguration()); + } + + + /** + * Gets the HCatTable schema for the table specified in the HCatInputFormat.setInput call + * on the specified job context. This information is available only after HCatInputFormat.setInput + * has been called for a JobContext. + * @param conf the Configuration object + * @return the table schema + * @throws IOException if HCatInputFormat.setInput has not been called + * for the current context + */ + public static HCatSchema getTableSchema(Configuration conf) + throws IOException { + InputJobInfo inputJobInfo = getJobInfo(conf); + HCatSchema allCols = new HCatSchema(new LinkedList()); + for (HCatFieldSchema field : + inputJobInfo.getTableInfo().getDataColumns().getFields()) + allCols.append(field); + for (HCatFieldSchema field : + inputJobInfo.getTableInfo().getPartitionColumns().getFields()) + allCols.append(field); + return allCols; + } + + /** + * Gets the InputJobInfo object by reading the Configuration and deserializing + * the string. If InputJobInfo is not present in the configuration, throws an + * exception since that means HCatInputFormat.setInput has not been called. + * @param conf the Configuration object + * @return the InputJobInfo object + * @throws IOException the exception + */ + private static InputJobInfo getJobInfo(Configuration conf) + throws IOException { + String jobString = conf.get( + HCatConstants.HCAT_KEY_JOB_INFO); + if (jobString == null) { + throw new IOException("job information not found in JobContext." + + " HCatInputFormat.setInput() not called?"); + } + + return (InputJobInfo) HCatUtil.deserialize(jobString); + } + + private void setInputPath(JobConf jobConf, String location) + throws IOException { + + // ideally we should just call FileInputFormat.setInputPaths() here - but + // that won't work since FileInputFormat.setInputPaths() needs + // a Job object instead of a JobContext which we are handed here + + int length = location.length(); + int curlyOpen = 0; + int pathStart = 0; + boolean globPattern = false; + List pathStrings = new ArrayList(); + + for (int i = 0; i < length; i++) { + char ch = location.charAt(i); + switch (ch) { + case '{': { + curlyOpen++; + if (!globPattern) { + globPattern = true; + } + break; + } + case '}': { + curlyOpen--; + if (curlyOpen == 0 && globPattern) { + globPattern = false; + } + break; + } + case ',': { + if (!globPattern) { + pathStrings.add(location.substring(pathStart, i)); + pathStart = i + 1; + } + break; + } + } + } + pathStrings.add(location.substring(pathStart, length)); + + Path[] paths = StringUtils.stringToPath(pathStrings.toArray(new String[0])); + String separator = ""; + StringBuilder str = new StringBuilder(); + + for (Path path : paths) { + FileSystem fs = path.getFileSystem(jobConf); + final String qualifiedPath = fs.makeQualified(path).toString(); + str.append(separator) + .append(StringUtils.escapeString(qualifiedPath)); + separator = StringUtils.COMMA_STR; + } + + jobConf.set("mapred.input.dir", str.toString()); + } + +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/HCatBaseOutputFormat.java hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/HCatBaseOutputFormat.java new file mode 100644 index 0000000..f8b6118 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/HCatBaseOutputFormat.java @@ -0,0 +1,246 @@ +/** + * 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.hcatalog.mapreduce; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.io.WritableComparable; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.OutputFormat; +import org.apache.hadoop.util.ReflectionUtils; +import org.apache.hcatalog.common.ErrorType; +import org.apache.hcatalog.common.HCatConstants; +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.common.HCatUtil; +import org.apache.hcatalog.data.HCatRecord; +import org.apache.hcatalog.data.schema.HCatSchema; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.HCatBaseOutputFormat} instead + */ +public abstract class HCatBaseOutputFormat extends OutputFormat, HCatRecord> { + +// static final private Log LOG = LogFactory.getLog(HCatBaseOutputFormat.class); + + /** + * @see org.apache.hcatalog.mapreduce.HCatBaseOutputFormat#getTableSchema(org.apache.hadoop.conf.Configuration) + * @deprecated Use {@link #getTableSchema(org.apache.hadoop.conf.Configuration)} + */ + public static HCatSchema getTableSchema(JobContext context) throws IOException { + return getTableSchema(context.getConfiguration()); + } + + /** + * Gets the table schema for the table specified in the HCatOutputFormat.setOutput call + * on the specified job context. + * @param conf the Configuration object + * @return the table schema + * @throws IOException if HCatOutputFormat.setOutput has not been called for the passed context + */ + public static HCatSchema getTableSchema(Configuration conf) throws IOException { + OutputJobInfo jobInfo = getJobInfo(conf); + return jobInfo.getTableInfo().getDataColumns(); + } + + /** + * Check for validity of the output-specification for the job. + * @param context information about the job + * @throws IOException when output should not be attempted + */ + @Override + public void checkOutputSpecs(JobContext context + ) throws IOException, InterruptedException { + getOutputFormat(context).checkOutputSpecs(context); + } + + /** + * Gets the output format instance. + * @param context the job context + * @return the output format instance + * @throws IOException + */ + protected OutputFormat, HCatRecord> getOutputFormat(JobContext context) throws IOException { + OutputJobInfo jobInfo = getJobInfo(context); + HCatStorageHandler storageHandler = HCatUtil.getStorageHandler(context.getConfiguration(), jobInfo.getTableInfo().getStorerInfo()); + //why do we need this? + configureOutputStorageHandler(context); + return storageHandler.getOutputFormatContainer(ReflectionUtils.newInstance(storageHandler.getOutputFormatClass(), context.getConfiguration())); + } + + /** + * @see org.apache.hcatalog.mapreduce.HCatBaseOutputFormat#getJobInfo(org.apache.hadoop.conf.Configuration) + * @deprecated use {@link #getJobInfo(org.apache.hadoop.conf.Configuration)} + */ + public static OutputJobInfo getJobInfo(JobContext jobContext) throws IOException { + return getJobInfo(jobContext.getConfiguration()); + } + + /** + * Gets the HCatOuputJobInfo object by reading the Configuration and deserializing + * the string. If InputJobInfo is not present in the configuration, throws an + * exception since that means HCatOutputFormat.setOutput has not been called. + * @param conf the job Configuration object + * @return the OutputJobInfo object + * @throws IOException the IO exception + */ + public static OutputJobInfo getJobInfo(Configuration conf) throws IOException { + String jobString = conf.get(HCatConstants.HCAT_KEY_OUTPUT_INFO); + if (jobString == null) { + throw new HCatException(ErrorType.ERROR_NOT_INITIALIZED); + } + + return (OutputJobInfo) HCatUtil.deserialize(jobString); + } + + /** + * Configure the output storage handler + * @param jobContext the job context + * @throws IOException + */ + @SuppressWarnings("unchecked") + static void configureOutputStorageHandler( + JobContext jobContext) throws IOException { + configureOutputStorageHandler(jobContext, (List) null); + } + + /** + * Configure the output storage handler with allowing specification of missing dynamic partvals + * @param jobContext the job context + * @param dynamicPartVals + * @throws IOException + */ + @SuppressWarnings("unchecked") + static void configureOutputStorageHandler( + JobContext jobContext, List dynamicPartVals) throws IOException { + Configuration conf = jobContext.getConfiguration(); + try { + OutputJobInfo jobInfo = (OutputJobInfo) HCatUtil.deserialize(conf.get(HCatConstants.HCAT_KEY_OUTPUT_INFO)); + HCatStorageHandler storageHandler = HCatUtil.getStorageHandler(conf, jobInfo.getTableInfo().getStorerInfo()); + + Map partitionValues = jobInfo.getPartitionValues(); + String location = jobInfo.getLocation(); + + if (dynamicPartVals != null) { + // dynamic part vals specified + List dynamicPartKeys = jobInfo.getDynamicPartitioningKeys(); + if (dynamicPartVals.size() != dynamicPartKeys.size()) { + throw new HCatException(ErrorType.ERROR_INVALID_PARTITION_VALUES, + "Unable to configure dynamic partitioning for storage handler, mismatch between" + + " number of partition values obtained[" + dynamicPartVals.size() + + "] and number of partition values required[" + dynamicPartKeys.size() + "]"); + } + for (int i = 0; i < dynamicPartKeys.size(); i++) { + partitionValues.put(dynamicPartKeys.get(i), dynamicPartVals.get(i)); + } + +// // re-home location, now that we know the rest of the partvals +// Table table = jobInfo.getTableInfo().getTable(); +// +// List partitionCols = new ArrayList(); +// for(FieldSchema schema : table.getPartitionKeys()) { +// partitionCols.add(schema.getName()); +// } + jobInfo.setPartitionValues(partitionValues); + } + + HCatUtil.configureOutputStorageHandler(storageHandler, conf, jobInfo); + } catch (Exception e) { + if (e instanceof HCatException) { + throw (HCatException) e; + } else { + throw new HCatException(ErrorType.ERROR_INIT_STORAGE_HANDLER, e); + } + } + } + + /** + * Configure the output storage handler, with allowing specification + * of partvals from which it picks the dynamic partvals + * @param context the job context + * @param jobInfo the output job info + * @param fullPartSpec + * @throws IOException + */ + + protected static void configureOutputStorageHandler( + JobContext context, OutputJobInfo jobInfo, + Map fullPartSpec) throws IOException { + List dynamicPartKeys = jobInfo.getDynamicPartitioningKeys(); + if ((dynamicPartKeys == null) || (dynamicPartKeys.isEmpty())) { + configureOutputStorageHandler(context, (List) null); + } else { + List dynKeyVals = new ArrayList(); + for (String dynamicPartKey : dynamicPartKeys) { + dynKeyVals.add(fullPartSpec.get(dynamicPartKey)); + } + configureOutputStorageHandler(context, dynKeyVals); + } + } + + + protected static void setPartDetails(OutputJobInfo jobInfo, final HCatSchema schema, + Map partMap) throws HCatException, IOException { + List posOfPartCols = new ArrayList(); + List posOfDynPartCols = new ArrayList(); + + // If partition columns occur in data, we want to remove them. + // So, find out positions of partition columns in schema provided by user. + // We also need to update the output Schema with these deletions. + + // Note that, output storage handlers never sees partition columns in data + // or schema. + + HCatSchema schemaWithoutParts = new HCatSchema(schema.getFields()); + for (String partKey : partMap.keySet()) { + Integer idx; + if ((idx = schema.getPosition(partKey)) != null) { + posOfPartCols.add(idx); + schemaWithoutParts.remove(schema.get(partKey)); + } + } + + // Also, if dynamic partitioning is being used, we want to + // set appropriate list of columns for the columns to be dynamically specified. + // These would be partition keys too, so would also need to be removed from + // output schema and partcols + + if (jobInfo.isDynamicPartitioningUsed()) { + for (String partKey : jobInfo.getDynamicPartitioningKeys()) { + Integer idx; + if ((idx = schema.getPosition(partKey)) != null) { + posOfPartCols.add(idx); + posOfDynPartCols.add(idx); + schemaWithoutParts.remove(schema.get(partKey)); + } + } + } + + HCatUtil.validatePartitionSchema( + new Table(jobInfo.getTableInfo().getTable()), schemaWithoutParts); + jobInfo.setPosOfPartCols(posOfPartCols); + jobInfo.setPosOfDynPartCols(posOfDynPartCols); + jobInfo.setOutputSchema(schemaWithoutParts); + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/HCatEximInputFormat.java.broken hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/HCatEximInputFormat.java.broken new file mode 100644 index 0000000..71b9652 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/HCatEximInputFormat.java.broken @@ -0,0 +1,141 @@ +/** + * 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.hcatalog.mapreduce; + +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.ql.parse.EximUtil; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hcatalog.common.HCatConstants; +import org.apache.hcatalog.common.HCatUtil; +import org.apache.hcatalog.data.schema.HCatSchema; +import org.apache.hcatalog.data.schema.HCatSchemaUtils; + +/** The InputFormat to use to read data from HCat */ +public class HCatEximInputFormat extends HCatBaseInputFormat { + + /** + * Set the input to use for the Job. This queries the metadata file with + * the specified partition predicates, gets the matching partitions, puts + * the information in the conf object. The inputInfo object is updated with + * information needed in the client context + * + * @param job the job object + * @return two hcat schemas, for the table columns and the partition keys + * @throws IOException + * the exception in communicating with the metadata server + */ + public static List setInput(Job job, + String location, + Map partitionFilter) throws IOException { + FileSystem fs; + try { + fs = FileSystem.get(new URI(location), job.getConfiguration()); + } catch (URISyntaxException e) { + throw new IOException(e); + } + Path fromPath = new Path(location); + Path metadataPath = new Path(fromPath, "_metadata"); + try { + Map.Entry> tp = EximUtil + .readMetaData(fs, metadataPath); + org.apache.hadoop.hive.metastore.api.Table table = tp.getKey(); + InputJobInfo inputInfo = InputJobInfo.create(table.getDbName(), table.getTableName(),null,null,null); + List partCols = table.getPartitionKeys(); + List partInfoList = null; + if (partCols.size() > 0) { + List partColNames = new ArrayList(partCols.size()); + for (FieldSchema fsc : partCols) { + partColNames.add(fsc.getName()); + } + List partitions = tp.getValue(); + partInfoList = filterPartitions(partitionFilter, partitions, table.getPartitionKeys()); + } else { + partInfoList = new ArrayList(1); + HCatSchema schema = new HCatSchema(HCatUtil.getHCatFieldSchemaList(table.getSd().getCols())); + Map parameters = table.getParameters(); + String inputStorageDriverClass = null; + if (parameters.containsKey(HCatConstants.HCAT_ISD_CLASS)){ + inputStorageDriverClass = parameters.get(HCatConstants.HCAT_ISD_CLASS); + }else{ + throw new IOException("No input storage driver classname found, cannot read partition"); + } + Properties hcatProperties = new Properties(); + for (String key : parameters.keySet()){ + if (key.startsWith(InitializeInput.HCAT_KEY_PREFIX)){ + hcatProperties.put(key, parameters.get(key)); + } + } + PartInfo partInfo = new PartInfo(schema, inputStorageDriverClass, location + "/data", hcatProperties); + partInfoList.add(partInfo); + } + inputInfo.setPartitions(partInfoList); + inputInfo.setTableInfo(HCatTableInfo.valueOf(table)); + job.getConfiguration().set( + HCatConstants.HCAT_KEY_JOB_INFO, + HCatUtil.serialize(inputInfo)); + List rv = new ArrayList(2); + rv.add(HCatSchemaUtils.getHCatSchema(table.getSd().getCols())); + rv.add(HCatSchemaUtils.getHCatSchema(partCols)); + return rv; + } catch(SemanticException e) { + throw new IOException(e); + } + } + + private static List filterPartitions(Map partitionFilter, + List partitions, List partCols) throws IOException { + List partInfos = new LinkedList(); + for (Partition partition : partitions) { + boolean matches = true; + List partVals = partition.getValues(); + assert partCols.size() == partVals.size(); + Map partSpec = EximUtil.makePartSpec(partCols, partVals); + if (partitionFilter != null) { + for (Map.Entry constraint : partitionFilter.entrySet()) { + String partVal = partSpec.get(constraint.getKey()); + if ((partVal == null) || !partVal.equals(constraint.getValue())) { + matches = false; + break; + } + } + } + if (matches) { + PartInfo partInfo = InitializeInput.extractPartInfo(partition.getSd(), + partition.getParameters()); + partInfo.setPartitionValues(partSpec); + partInfos.add(partInfo); + } + } + return partInfos; + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/HCatEximOutputCommitter.java.broken hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/HCatEximOutputCommitter.java.broken new file mode 100644 index 0000000..0ab8c22 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/HCatEximOutputCommitter.java.broken @@ -0,0 +1,166 @@ +/** + * 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.hcatalog.mapreduce; + +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.metastore.Warehouse; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.parse.EximUtil; +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.JobStatus; +import org.apache.hadoop.mapreduce.OutputCommitter; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hcatalog.common.ErrorType; +import org.apache.hcatalog.common.HCatException; + +public class HCatEximOutputCommitter extends OutputCommitter { + + private static final Log LOG = LogFactory.getLog(HCatEximOutputCommitter.class); + + private final OutputCommitter baseCommitter; + + public HCatEximOutputCommitter(JobContext context, OutputCommitter baseCommitter) { + this.baseCommitter = baseCommitter; + } + + @Override + public void abortTask(TaskAttemptContext context) throws IOException { + baseCommitter.abortTask(context); + } + + @Override + public void commitTask(TaskAttemptContext context) throws IOException { + baseCommitter.commitTask(context); + } + + @Override + public boolean needsTaskCommit(TaskAttemptContext context) throws IOException { + return baseCommitter.needsTaskCommit(context); + } + + @Override + public void setupJob(JobContext context) throws IOException { + if( baseCommitter != null ) { + baseCommitter.setupJob(context); + } + } + + @Override + public void setupTask(TaskAttemptContext context) throws IOException { + baseCommitter.setupTask(context); + } + + @Override + public void abortJob(JobContext jobContext, JobStatus.State state) throws IOException { + if(baseCommitter != null) { + baseCommitter.abortJob(jobContext, state); + } + OutputJobInfo jobInfo = HCatOutputFormat.getJobInfo(jobContext); + + Path src = new Path(jobInfo.getLocation()); + FileSystem fs = src.getFileSystem(jobContext.getConfiguration()); + fs.delete(src, true); + } + + @Override + public void commitJob(JobContext jobContext) throws IOException { + if(baseCommitter != null) { + baseCommitter.commitJob(jobContext); + } + } + + @Override + public void cleanupJob(JobContext jobContext) throws IOException { + LOG.info("HCatEximOutputCommitter.cleanup invoked; m.o.d : " + + jobContext.getConfiguration().get("mapred.output.dir")); + if (baseCommitter != null) { + LOG.info("baseCommitter.class = " + baseCommitter.getClass().getName()); + baseCommitter.cleanupJob(jobContext); + } + + OutputJobInfo jobInfo = HCatBaseOutputFormat.getJobInfo(jobContext); + Configuration conf = jobContext.getConfiguration(); + FileSystem fs; + try { + fs = FileSystem.get(new URI(jobInfo.getTableInfo().getTable().getSd().getLocation()), conf); + } catch (URISyntaxException e) { + throw new IOException(e); + } + doCleanup(jobInfo, fs); + } + + private static void doCleanup(OutputJobInfo jobInfo, FileSystem fs) throws IOException, + HCatException { + try { + Table ttable = jobInfo.getTableInfo().getTable(); + org.apache.hadoop.hive.ql.metadata.Table table = new org.apache.hadoop.hive.ql.metadata.Table( + ttable); + StorageDescriptor tblSD = ttable.getSd(); + Path tblPath = new Path(tblSD.getLocation()); + Path path = new Path(tblPath, "_metadata"); + List tpartitions = null; + try { + Map.Entry> rv = EximUtil + .readMetaData(fs, path); + tpartitions = rv.getValue(); + } catch (IOException e) { + } + List partitions = + new ArrayList(); + if (tpartitions != null) { + for (Partition tpartition : tpartitions) { + partitions.add(new org.apache.hadoop.hive.ql.metadata.Partition(table, tpartition)); + } + } + if (!table.getPartitionKeys().isEmpty()) { + Map partitionValues = jobInfo.getPartitionValues(); + org.apache.hadoop.hive.ql.metadata.Partition partition = + new org.apache.hadoop.hive.ql.metadata.Partition(table, + partitionValues, + new Path(tblPath, Warehouse.makePartPath(partitionValues))); + partition.getTPartition().setParameters(table.getParameters()); + partitions.add(partition); + } + EximUtil.createExportDump(fs, path, (table), partitions); + } catch (SemanticException e) { + throw new HCatException(ErrorType.ERROR_PUBLISHING_PARTITION, e); + } catch (HiveException e) { + throw new HCatException(ErrorType.ERROR_PUBLISHING_PARTITION, e); + } catch (MetaException e) { + throw new HCatException(ErrorType.ERROR_PUBLISHING_PARTITION, e); + } + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/HCatEximOutputFormat.java.broken hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/HCatEximOutputFormat.java.broken new file mode 100644 index 0000000..6181284 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/HCatEximOutputFormat.java.broken @@ -0,0 +1,176 @@ +/** + * 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.hcatalog.mapreduce; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.TreeMap; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.metastore.Warehouse; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.SerDeInfo; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.ql.io.RCFileInputFormat; +import org.apache.hadoop.hive.ql.io.RCFileOutputFormat; +import org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe; +import org.apache.hadoop.io.WritableComparable; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.OutputCommitter; +import org.apache.hadoop.mapreduce.RecordWriter; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hcatalog.common.ErrorType; +import org.apache.hcatalog.common.HCatConstants; +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.common.HCatUtil; +import org.apache.hcatalog.data.HCatRecord; +import org.apache.hcatalog.data.schema.HCatFieldSchema; +import org.apache.hcatalog.data.schema.HCatSchema; +import org.apache.hcatalog.data.schema.HCatSchemaUtils; +import org.apache.hcatalog.rcfile.RCFileInputDriver; +import org.apache.hcatalog.rcfile.RCFileOutputDriver; + +/** + * The OutputFormat to use to write data to HCat without a hcat server. This can then + * be imported into a hcat instance, or used with a HCatEximInputFormat. As in + * HCatOutputFormat, the key value is ignored and + * and should be given as null. The value is the HCatRecord to write. + */ +public class HCatEximOutputFormat extends HCatBaseOutputFormat { + + private static final Log LOG = LogFactory.getLog(HCatEximOutputFormat.class); + + /** + * Get the record writer for the job. Uses the Table's default OutputStorageDriver + * to get the record writer. + * + * @param context + * the information about the current task. + * @return a RecordWriter to write the output for the job. + * @throws IOException + */ + @Override + public RecordWriter, HCatRecord> + getRecordWriter(TaskAttemptContext context + ) throws IOException, InterruptedException { + return getOutputFormat(context).getRecordWriter(context); + } + + /** + * Get the output committer for this output format. This is responsible + * for ensuring the output is committed correctly. + * @param context the task context + * @return an output committer + * @throws IOException + * @throws InterruptedException + */ + @Override + public OutputCommitter getOutputCommitter(TaskAttemptContext context) throws IOException, InterruptedException { + return new HCatEximOutputCommitter(context,((OutputCommitterContainer)getOutputFormat(context).getOutputCommitter(context)).getBaseOutputCommitter()); + } + + /** + * Check for validity of the output-specification for the job. + * @param context information about the job + * @throws IOException when output should not be attempted + */ + @Override + public void checkOutputSpecs(JobContext context + ) throws IOException, InterruptedException { + ((OutputFormatContainer)getOutputFormat(context)).getBaseOutputFormat().checkOutputSpecs(context); + } + + public static void setOutput(Job job, String dbname, String tablename, String location, + HCatSchema partitionSchema, List partitionValues, HCatSchema columnSchema) throws HCatException { + setOutput(job, dbname, tablename, location, partitionSchema, partitionValues, columnSchema, + RCFileInputDriver.class.getName(), + RCFileOutputDriver.class.getName(), + RCFileInputFormat.class.getName(), + RCFileOutputFormat.class.getName(), + ColumnarSerDe.class.getName()); + } + + @SuppressWarnings("unchecked") + public static void setOutput(Job job, String dbname, String tablename, String location, + HCatSchema partitionSchema, + List partitionValues, + HCatSchema columnSchema, + String isdname, String osdname, + String ifname, String ofname, + String serializationLib) throws HCatException { + Map partSpec = new TreeMap(); + List partKeys = null; + if (partitionSchema != null) { + partKeys = partitionSchema.getFields(); + if (partKeys.size() != partitionValues.size()) { + throw new IllegalArgumentException("Partition key size differs from partition value size"); + } + for (int i = 0; i < partKeys.size(); ++i) { + HCatFieldSchema partKey = partKeys.get(i); + if (partKey.getType() != HCatFieldSchema.Type.STRING) { + throw new IllegalArgumentException("Partition key type string is only supported"); + } + partSpec.put(partKey.getName(), partitionValues.get(i)); + } + } + StorerInfo storerInfo = new StorerInfo(isdname, osdname, new Properties()); + OutputJobInfo outputJobInfo = OutputJobInfo.create(dbname,tablename,partSpec,null,null); + org.apache.hadoop.hive.ql.metadata.Table tbl = new + org.apache.hadoop.hive.ql.metadata.Table(dbname, tablename); + Table table = tbl.getTTable(); + table.getParameters().put(HCatConstants.HCAT_ISD_CLASS, isdname); + table.getParameters().put(HCatConstants.HCAT_OSD_CLASS, osdname); + try { + String partname = null; + if ((partKeys != null) && !partKeys.isEmpty()) { + List partSchema = HCatSchemaUtils.getFieldSchemas(partKeys); + table.setPartitionKeys(partSchema); + partname = Warehouse.makePartName(partSchema, partitionValues); + } else { + partname = "data"; + } + StorageDescriptor sd = table.getSd(); + sd.setLocation(location); + String dataLocation = location + "/" + partname; + outputJobInfo.setTableInfo(new HCatTableInfo(dbname,tablename,columnSchema,null,storerInfo,table)); + outputJobInfo.setOutputSchema(columnSchema); + outputJobInfo.setLocation(dataLocation); + setPartDetails(outputJobInfo, columnSchema, partSpec); + sd.setCols(HCatUtil.getFieldSchemaList(outputJobInfo.getOutputSchema().getFields())); + sd.setInputFormat(ifname); + sd.setOutputFormat(ofname); + SerDeInfo serdeInfo = sd.getSerdeInfo(); + serdeInfo.setSerializationLib(serializationLib); + Configuration conf = job.getConfiguration(); + conf.set(HCatConstants.HCAT_KEY_OUTPUT_INFO, HCatUtil.serialize(outputJobInfo)); + } catch (IOException e) { + throw new HCatException(ErrorType.ERROR_SET_OUTPUT, e); + } catch (MetaException e) { + throw new HCatException(ErrorType.ERROR_SET_OUTPUT, e); + } + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/HCatInputFormat.java hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/HCatInputFormat.java new file mode 100644 index 0000000..fd80e86 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/HCatInputFormat.java @@ -0,0 +1,137 @@ +/** + * 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.hcatalog.mapreduce; + +import java.io.IOException; +import java.util.Properties; + +import com.google.common.base.Preconditions; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.mapreduce.Job; + +/** + * The InputFormat to use to read data from HCatalog. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.HCatInputFormat} instead + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public class HCatInputFormat extends HCatBaseInputFormat { + + private Configuration conf; + private InputJobInfo inputJobInfo; + + /** + * @deprecated as of release 0.5, and will be removed in a future release + */ + @Deprecated + public static void setInput(Job job, InputJobInfo inputJobInfo) throws IOException { + setInput(job.getConfiguration(), inputJobInfo); + } + + /** + * @deprecated as of release 0.5, and will be removed in a future release + */ + @Deprecated + public static void setInput(Configuration conf, InputJobInfo inputJobInfo) throws IOException { + setInput(conf, inputJobInfo.getDatabaseName(), inputJobInfo.getTableName()) + .setFilter(inputJobInfo.getFilter()) + .setProperties(inputJobInfo.getProperties()); + } + + /** + * See {@link #setInput(org.apache.hadoop.conf.Configuration, String, String)} + */ + public static HCatInputFormat setInput(Job job, String dbName, String tableName) throws IOException { + return setInput(job.getConfiguration(), dbName, tableName); + } + + /** + * Set inputs to use for the job. This queries the metastore with the given input + * specification and serializes matching partitions into the job conf for use by MR tasks. + * @param conf the job configuration + * @param dbName database name, which if null 'default' is used + * @param tableName table name + * @throws IOException on all errors + */ + public static HCatInputFormat setInput(Configuration conf, String dbName, String tableName) + throws IOException { + + Preconditions.checkNotNull(conf, "required argument 'conf' is null"); + Preconditions.checkNotNull(tableName, "required argument 'tableName' is null"); + + HCatInputFormat hCatInputFormat = new HCatInputFormat(); + hCatInputFormat.conf = conf; + hCatInputFormat.inputJobInfo = InputJobInfo.create(dbName, tableName, null, null); + + try { + InitializeInput.setInput(conf, hCatInputFormat.inputJobInfo); + } catch (Exception e) { + throw new IOException(e); + } + + return hCatInputFormat; + } + + /** + * Set a filter on the input table. + * @param filter the filter specification, which may be null + * @return this + * @throws IOException on all errors + */ + public HCatInputFormat setFilter(String filter) throws IOException { + // null filters are supported to simplify client code + if (filter != null) { + inputJobInfo = InputJobInfo.create( + inputJobInfo.getDatabaseName(), + inputJobInfo.getTableName(), + filter, + inputJobInfo.getProperties()); + try { + InitializeInput.setInput(conf, inputJobInfo); + } catch (Exception e) { + throw new IOException(e); + } + } + return this; + } + + /** + * Set properties for the input format. + * @param properties properties for the input specification + * @return this + * @throws IOException on all errors + */ + public HCatInputFormat setProperties(Properties properties) throws IOException { + Preconditions.checkNotNull(properties, "required argument 'properties' is null"); + inputJobInfo = InputJobInfo.create( + inputJobInfo.getDatabaseName(), + inputJobInfo.getTableName(), + inputJobInfo.getFilter(), + properties); + try { + InitializeInput.setInput(conf, inputJobInfo); + } catch (Exception e) { + throw new IOException(e); + } + return this; + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/HCatOutputFormat.java hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/HCatOutputFormat.java new file mode 100644 index 0000000..1ee7e9b --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/HCatOutputFormat.java @@ -0,0 +1,280 @@ +/** + * 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.hcatalog.mapreduce; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.Index; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.io.WritableComparable; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.OutputCommitter; +import org.apache.hadoop.mapreduce.RecordWriter; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.security.Credentials; +import org.apache.hcatalog.common.ErrorType; +import org.apache.hcatalog.common.HCatConstants; +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.common.HCatUtil; +import org.apache.hcatalog.data.HCatRecord; +import org.apache.hcatalog.data.schema.HCatSchema; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** The OutputFormat to use to write data to HCatalog. The key value is ignored and + * should be given as null. The value is the HCatRecord to write. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.HCatOutputFormat} instead + */ +public class HCatOutputFormat extends HCatBaseOutputFormat { + + static final private Logger LOG = LoggerFactory.getLogger(HCatOutputFormat.class); + + private static int maxDynamicPartitions; + private static boolean harRequested; + + /** + * @see org.apache.hcatalog.mapreduce.HCatOutputFormat#setOutput(org.apache.hadoop.conf.Configuration, Credentials, OutputJobInfo) + */ + public static void setOutput(Job job, OutputJobInfo outputJobInfo) throws IOException { + setOutput(job.getConfiguration(), job.getCredentials(), outputJobInfo); + } + + /** + * Set the information about the output to write for the job. This queries the metadata server + * to find the StorageHandler to use for the table. It throws an error if the + * partition is already published. + * @param conf the Configuration object + * @param credentials the Credentials object + * @param outputJobInfo the table output information for the job + * @throws IOException the exception in communicating with the metadata server + */ + @SuppressWarnings("unchecked") + public static void setOutput(Configuration conf, Credentials credentials, + OutputJobInfo outputJobInfo) throws IOException { + HiveMetaStoreClient client = null; + + try { + + HiveConf hiveConf = HCatUtil.getHiveConf(conf); + client = HCatUtil.getHiveClient(hiveConf); + Table table = HCatUtil.getTable(client, outputJobInfo.getDatabaseName(), + outputJobInfo.getTableName()); + + List indexList = client.listIndexNames(outputJobInfo.getDatabaseName(), outputJobInfo.getTableName(), Short.MAX_VALUE); + + for (String indexName : indexList) { + Index index = client.getIndex(outputJobInfo.getDatabaseName(), outputJobInfo.getTableName(), indexName); + if (!index.isDeferredRebuild()) { + throw new HCatException(ErrorType.ERROR_NOT_SUPPORTED, "Store into a table with an automatic index from Pig/Mapreduce is not supported"); + } + } + StorageDescriptor sd = table.getTTable().getSd(); + + if (sd.isCompressed()) { + throw new HCatException(ErrorType.ERROR_NOT_SUPPORTED, "Store into a compressed partition from Pig/Mapreduce is not supported"); + } + + if (sd.getBucketCols() != null && !sd.getBucketCols().isEmpty()) { + throw new HCatException(ErrorType.ERROR_NOT_SUPPORTED, "Store into a partition with bucket definition from Pig/Mapreduce is not supported"); + } + + if (sd.getSortCols() != null && !sd.getSortCols().isEmpty()) { + throw new HCatException(ErrorType.ERROR_NOT_SUPPORTED, "Store into a partition with sorted column definition from Pig/Mapreduce is not supported"); + } + + if (table.getTTable().getPartitionKeysSize() == 0) { + if ((outputJobInfo.getPartitionValues() != null) && (!outputJobInfo.getPartitionValues().isEmpty())) { + // attempt made to save partition values in non-partitioned table - throw error. + throw new HCatException(ErrorType.ERROR_INVALID_PARTITION_VALUES, + "Partition values specified for non-partitioned table"); + } + // non-partitioned table + outputJobInfo.setPartitionValues(new HashMap()); + + } else { + // partitioned table, we expect partition values + // convert user specified map to have lower case key names + Map valueMap = new HashMap(); + if (outputJobInfo.getPartitionValues() != null) { + for (Map.Entry entry : outputJobInfo.getPartitionValues().entrySet()) { + valueMap.put(entry.getKey().toLowerCase(), entry.getValue()); + } + } + + if ((outputJobInfo.getPartitionValues() == null) + || (outputJobInfo.getPartitionValues().size() < table.getTTable().getPartitionKeysSize())) { + // dynamic partition usecase - partition values were null, or not all were specified + // need to figure out which keys are not specified. + List dynamicPartitioningKeys = new ArrayList(); + boolean firstItem = true; + for (FieldSchema fs : table.getPartitionKeys()) { + if (!valueMap.containsKey(fs.getName().toLowerCase())) { + dynamicPartitioningKeys.add(fs.getName().toLowerCase()); + } + } + + if (valueMap.size() + dynamicPartitioningKeys.size() != table.getTTable().getPartitionKeysSize()) { + // If this isn't equal, then bogus key values have been inserted, error out. + throw new HCatException(ErrorType.ERROR_INVALID_PARTITION_VALUES, "Invalid partition keys specified"); + } + + outputJobInfo.setDynamicPartitioningKeys(dynamicPartitioningKeys); + String dynHash; + if ((dynHash = conf.get(HCatConstants.HCAT_DYNAMIC_PTN_JOBID)) == null) { + dynHash = String.valueOf(Math.random()); +// LOG.info("New dynHash : ["+dynHash+"]"); +// }else{ +// LOG.info("Old dynHash : ["+dynHash+"]"); + } + conf.set(HCatConstants.HCAT_DYNAMIC_PTN_JOBID, dynHash); + + } + + outputJobInfo.setPartitionValues(valueMap); + } + + HCatSchema tableSchema = HCatUtil.extractSchema(table); + StorerInfo storerInfo = + InternalUtil.extractStorerInfo(table.getTTable().getSd(), table.getParameters()); + + List partitionCols = new ArrayList(); + for (FieldSchema schema : table.getPartitionKeys()) { + partitionCols.add(schema.getName()); + } + + HCatStorageHandler storageHandler = HCatUtil.getStorageHandler(conf, storerInfo); + + //Serialize the output info into the configuration + outputJobInfo.setTableInfo(HCatTableInfo.valueOf(table.getTTable())); + outputJobInfo.setOutputSchema(tableSchema); + harRequested = getHarRequested(hiveConf); + outputJobInfo.setHarRequested(harRequested); + maxDynamicPartitions = getMaxDynamicPartitions(hiveConf); + outputJobInfo.setMaximumDynamicPartitions(maxDynamicPartitions); + + HCatUtil.configureOutputStorageHandler(storageHandler, conf, outputJobInfo); + + Path tblPath = new Path(table.getTTable().getSd().getLocation()); + + /* Set the umask in conf such that files/dirs get created with table-dir + * permissions. Following three assumptions are made: + * 1. Actual files/dirs creation is done by RecordWriter of underlying + * output format. It is assumed that they use default permissions while creation. + * 2. Default Permissions = FsPermission.getDefault() = 777. + * 3. UMask is honored by underlying filesystem. + */ + + FsPermission.setUMask(conf, FsPermission.getDefault().applyUMask( + tblPath.getFileSystem(conf).getFileStatus(tblPath).getPermission())); + + if (Security.getInstance().isSecurityEnabled()) { + Security.getInstance().handleSecurity(credentials, outputJobInfo, client, conf, harRequested); + } + } catch (Exception e) { + if (e instanceof HCatException) { + throw (HCatException) e; + } else { + throw new HCatException(ErrorType.ERROR_SET_OUTPUT, e); + } + } finally { + HCatUtil.closeHiveClientQuietly(client); + } + } + + /** + * @see org.apache.hcatalog.mapreduce.HCatOutputFormat#setSchema(org.apache.hadoop.conf.Configuration, org.apache.hcatalog.data.schema.HCatSchema) + */ + public static void setSchema(final Job job, final HCatSchema schema) throws IOException { + setSchema(job.getConfiguration(), schema); + } + + /** + * Set the schema for the data being written out to the partition. The + * table schema is used by default for the partition if this is not called. + * @param conf the job Configuration object + * @param schema the schema for the data + * @throws IOException + */ + public static void setSchema(final Configuration conf, final HCatSchema schema) throws IOException { + OutputJobInfo jobInfo = getJobInfo(conf); + Map partMap = jobInfo.getPartitionValues(); + setPartDetails(jobInfo, schema, partMap); + conf.set(HCatConstants.HCAT_KEY_OUTPUT_INFO, HCatUtil.serialize(jobInfo)); + } + + /** + * Get the record writer for the job. This uses the StorageHandler's default + * OutputFormat to get the record writer. + * @param context the information about the current task + * @return a RecordWriter to write the output for the job + * @throws IOException + * @throws InterruptedException + */ + @Override + public RecordWriter, HCatRecord> + getRecordWriter(TaskAttemptContext context) + throws IOException, InterruptedException { + return getOutputFormat(context).getRecordWriter(context); + } + + + /** + * Get the output committer for this output format. This is responsible + * for ensuring the output is committed correctly. + * @param context the task context + * @return an output committer + * @throws IOException + * @throws InterruptedException + */ + @Override + public OutputCommitter getOutputCommitter(TaskAttemptContext context + ) throws IOException, InterruptedException { + return getOutputFormat(context).getOutputCommitter(context); + } + + private static int getMaxDynamicPartitions(HiveConf hConf) { + // by default the bounds checking for maximum number of + // dynamic partitions is disabled (-1) + int maxDynamicPartitions = -1; + + if (HCatConstants.HCAT_IS_DYNAMIC_MAX_PTN_CHECK_ENABLED) { + maxDynamicPartitions = hConf.getIntVar( + HiveConf.ConfVars.DYNAMICPARTITIONMAXPARTS); + } + + return maxDynamicPartitions; + } + + private static boolean getHarRequested(HiveConf hConf) { + return hConf.getBoolVar(HiveConf.ConfVars.HIVEARCHIVEENABLED); + } + +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/HCatRecordReader.java hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/HCatRecordReader.java new file mode 100644 index 0000000..5c1a4dc --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/HCatRecordReader.java @@ -0,0 +1,286 @@ +/** + * 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.hcatalog.mapreduce; + +import java.io.IOException; +import java.util.Map; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.serde2.Deserializer; +import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.io.WritableComparable; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapreduce.RecordReader; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.util.ReflectionUtils; +import org.apache.hcatalog.common.HCatConstants; +import org.apache.hcatalog.common.HCatUtil; +import org.apache.hcatalog.data.DefaultHCatRecord; +import org.apache.hcatalog.data.HCatRecord; +import org.apache.hcatalog.data.LazyHCatRecord; +import org.apache.hcatalog.data.schema.HCatSchema; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** The HCat wrapper for the underlying RecordReader, + * this ensures that the initialize on + * the underlying record reader is done with the underlying split, + * not with HCatSplit. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.HCatRecordReader} instead + */ +class HCatRecordReader extends RecordReader { + + private static final Logger LOG = LoggerFactory.getLogger(HCatRecordReader.class); + + private InputErrorTracker errorTracker; + + WritableComparable currentKey; + Writable currentValue; + HCatRecord currentHCatRecord; + + /** The underlying record reader to delegate to. */ + private org.apache.hadoop.mapred.RecordReader baseRecordReader; + + /** The storage handler used */ + private final HCatStorageHandler storageHandler; + + private Deserializer deserializer; + + private Map valuesNotInDataCols; + + private HCatSchema outputSchema = null; + private HCatSchema dataSchema = null; + + /** + * Instantiates a new hcat record reader. + */ + public HCatRecordReader(HCatStorageHandler storageHandler, + Map valuesNotInDataCols) { + this.storageHandler = storageHandler; + this.valuesNotInDataCols = valuesNotInDataCols; + } + + /* (non-Javadoc) + * @see org.apache.hadoop.mapreduce.RecordReader#initialize( + * org.apache.hadoop.mapreduce.InputSplit, + * org.apache.hadoop.mapreduce.TaskAttemptContext) + */ + @Override + public void initialize(org.apache.hadoop.mapreduce.InputSplit split, + TaskAttemptContext taskContext) throws IOException, InterruptedException { + + HCatSplit hcatSplit = InternalUtil.castToHCatSplit(split); + + baseRecordReader = createBaseRecordReader(hcatSplit, storageHandler, taskContext); + createDeserializer(hcatSplit, storageHandler, taskContext); + + // Pull the output schema out of the TaskAttemptContext + outputSchema = (HCatSchema) HCatUtil.deserialize( + taskContext.getConfiguration().get(HCatConstants.HCAT_KEY_OUTPUT_SCHEMA)); + + if (outputSchema == null) { + outputSchema = hcatSplit.getTableSchema(); + } + + // Pull the table schema out of the Split info + // TODO This should be passed in the TaskAttemptContext instead + dataSchema = hcatSplit.getDataSchema(); + + errorTracker = new InputErrorTracker(taskContext.getConfiguration()); + } + + private org.apache.hadoop.mapred.RecordReader createBaseRecordReader(HCatSplit hcatSplit, + HCatStorageHandler storageHandler, TaskAttemptContext taskContext) throws IOException { + + JobConf jobConf = HCatUtil.getJobConfFromContext(taskContext); + HCatUtil.copyJobPropertiesToJobConf(hcatSplit.getPartitionInfo().getJobProperties(), jobConf); + org.apache.hadoop.mapred.InputFormat inputFormat = + HCatInputFormat.getMapRedInputFormat(jobConf, storageHandler.getInputFormatClass()); + return inputFormat.getRecordReader(hcatSplit.getBaseSplit(), jobConf, + InternalUtil.createReporter(taskContext)); + } + + private void createDeserializer(HCatSplit hcatSplit, HCatStorageHandler storageHandler, + TaskAttemptContext taskContext) throws IOException { + + deserializer = ReflectionUtils.newInstance(storageHandler.getSerDeClass(), + taskContext.getConfiguration()); + + try { + InternalUtil.initializeDeserializer(deserializer, storageHandler.getConf(), + hcatSplit.getPartitionInfo().getTableInfo(), + hcatSplit.getPartitionInfo().getPartitionSchema()); + } catch (SerDeException e) { + throw new IOException("Failed initializing deserializer " + + storageHandler.getSerDeClass().getName(), e); + } + } + + /* (non-Javadoc) + * @see org.apache.hadoop.mapreduce.RecordReader#getCurrentKey() + */ + @Override + public WritableComparable getCurrentKey() + throws IOException, InterruptedException { + return currentKey; + } + + /* (non-Javadoc) + * @see org.apache.hadoop.mapreduce.RecordReader#getCurrentValue() + */ + @Override + public HCatRecord getCurrentValue() throws IOException, InterruptedException { + return currentHCatRecord; + } + + /* (non-Javadoc) + * @see org.apache.hadoop.mapreduce.RecordReader#getProgress() + */ + @Override + public float getProgress() { + try { + return baseRecordReader.getProgress(); + } catch (IOException e) { + LOG.warn("Exception in HCatRecord reader", e); + } + return 0.0f; // errored + } + + /** + * Check if the wrapped RecordReader has another record, and if so convert it into an + * HCatRecord. We both check for records and convert here so a configurable percent of + * bad records can be tolerated. + * + * @return if there is a next record + * @throws IOException on error + * @throws InterruptedException on error + */ + @Override + public boolean nextKeyValue() throws IOException, InterruptedException { + if (currentKey == null) { + currentKey = baseRecordReader.createKey(); + currentValue = baseRecordReader.createValue(); + } + + while (baseRecordReader.next(currentKey, currentValue)) { + HCatRecord r = null; + Throwable t = null; + + errorTracker.incRecords(); + + try { + Object o = deserializer.deserialize(currentValue); + r = new LazyHCatRecord(o, deserializer.getObjectInspector()); + } catch (Throwable throwable) { + t = throwable; + } + + if (r == null) { + errorTracker.incErrors(t); + continue; + } + + DefaultHCatRecord dr = new DefaultHCatRecord(outputSchema.size()); + int i = 0; + for (String fieldName : outputSchema.getFieldNames()) { + if (dataSchema.getPosition(fieldName) != null) { + dr.set(i, r.get(fieldName, dataSchema)); + } else { + dr.set(i, valuesNotInDataCols.get(fieldName)); + } + i++; + } + + currentHCatRecord = dr; + return true; + } + + return false; + } + + /* (non-Javadoc) + * @see org.apache.hadoop.mapreduce.RecordReader#close() + */ + @Override + public void close() throws IOException { + baseRecordReader.close(); + } + + /** + * Tracks number of of errors in input and throws a Runtime exception + * if the rate of errors crosses a limit. + *
+ * The intention is to skip over very rare file corruption or incorrect + * input, but catch programmer errors (incorrect format, or incorrect + * deserializers etc). + * + * This class was largely copied from Elephant-Bird (thanks @rangadi!) + * https://github.com/kevinweil/elephant-bird/blob/master/core/src/main/java/com/twitter/elephantbird/mapreduce/input/LzoRecordReader.java + */ + static class InputErrorTracker { + long numRecords; + long numErrors; + + double errorThreshold; // max fraction of errors allowed + long minErrors; // throw error only after this many errors + + InputErrorTracker(Configuration conf) { + errorThreshold = conf.getFloat(HCatConstants.HCAT_INPUT_BAD_RECORD_THRESHOLD_KEY, + HCatConstants.HCAT_INPUT_BAD_RECORD_THRESHOLD_DEFAULT); + minErrors = conf.getLong(HCatConstants.HCAT_INPUT_BAD_RECORD_MIN_KEY, + HCatConstants.HCAT_INPUT_BAD_RECORD_MIN_DEFAULT); + numRecords = 0; + numErrors = 0; + } + + void incRecords() { + numRecords++; + } + + void incErrors(Throwable cause) { + numErrors++; + if (numErrors > numRecords) { + // incorrect use of this class + throw new RuntimeException("Forgot to invoke incRecords()?"); + } + + if (cause == null) { + cause = new Exception("Unknown error"); + } + + if (errorThreshold <= 0) { // no errors are tolerated + throw new RuntimeException("error while reading input records", cause); + } + + LOG.warn("Error while reading an input record (" + + numErrors + " out of " + numRecords + " so far ): ", cause); + + double errRate = numErrors / (double) numRecords; + + // will always excuse the first error. We can decide if single + // error crosses threshold inside close() if we want to. + if (numErrors >= minErrors && errRate > errorThreshold) { + LOG.error(numErrors + " out of " + numRecords + + " crosses configured threshold (" + errorThreshold + ")"); + throw new RuntimeException("error rate while reading input records crossed threshold", cause); + } + } + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/HCatSplit.java hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/HCatSplit.java new file mode 100644 index 0000000..660320e --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/HCatSplit.java @@ -0,0 +1,189 @@ +/** + * 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.hcatalog.mapreduce; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.lang.reflect.Constructor; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.io.WritableUtils; +import org.apache.hadoop.mapreduce.InputSplit; +import org.apache.hcatalog.common.HCatUtil; +import org.apache.hcatalog.data.schema.HCatSchema; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The HCatSplit wrapper around the InputSplit returned by the underlying InputFormat + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.HCatSplit} instead + */ +public class HCatSplit extends InputSplit + implements Writable, org.apache.hadoop.mapred.InputSplit { + + private static final Logger LOG = LoggerFactory.getLogger(HCatSplit.class); + /** The partition info for the split. */ + private PartInfo partitionInfo; + + /** The split returned by the underlying InputFormat split. */ + private org.apache.hadoop.mapred.InputSplit baseMapRedSplit; + + /** The schema for the HCatTable */ + private HCatSchema tableSchema; + + private HiveConf hiveConf; + + /** + * Instantiates a new hcat split. + */ + public HCatSplit() { + } + + /** + * Instantiates a new hcat split. + * + * @param partitionInfo the partition info + * @param baseMapRedSplit the base mapred split + * @param tableSchema the table level schema + */ + public HCatSplit(PartInfo partitionInfo, + org.apache.hadoop.mapred.InputSplit baseMapRedSplit, + HCatSchema tableSchema) { + + this.partitionInfo = partitionInfo; + // dataSchema can be obtained from partitionInfo.getPartitionSchema() + this.baseMapRedSplit = baseMapRedSplit; + this.tableSchema = tableSchema; + } + + /** + * Gets the partition info. + * @return the partitionInfo + */ + public PartInfo getPartitionInfo() { + return partitionInfo; + } + + /** + * Gets the underlying InputSplit. + * @return the baseMapRedSplit + */ + public org.apache.hadoop.mapred.InputSplit getBaseSplit() { + return baseMapRedSplit; + } + + /** + * Gets the data schema. + * @return the table schema + */ + public HCatSchema getDataSchema() { + return this.partitionInfo.getPartitionSchema(); + } + + /** + * Gets the table schema. + * @return the table schema + */ + public HCatSchema getTableSchema() { + return this.tableSchema; + } + + /* (non-Javadoc) + * @see org.apache.hadoop.mapreduce.InputSplit#getLength() + */ + @Override + public long getLength() { + try { + return baseMapRedSplit.getLength(); + } catch (IOException e) { + LOG.warn("Exception in HCatSplit", e); + } + return 0; // we errored + } + + /* (non-Javadoc) + * @see org.apache.hadoop.mapreduce.InputSplit#getLocations() + */ + @Override + public String[] getLocations() { + try { + return baseMapRedSplit.getLocations(); + } catch (IOException e) { + LOG.warn("Exception in HCatSplit", e); + } + return new String[0]; // we errored + } + + /* (non-Javadoc) + * @see org.apache.hadoop.io.Writable#readFields(java.io.DataInput) + */ + @SuppressWarnings("unchecked") + @Override + public void readFields(DataInput input) throws IOException { + String partitionInfoString = WritableUtils.readString(input); + partitionInfo = (PartInfo) HCatUtil.deserialize(partitionInfoString); + + String baseSplitClassName = WritableUtils.readString(input); + org.apache.hadoop.mapred.InputSplit split; + try { + Class splitClass = + (Class) Class.forName(baseSplitClassName); + + //Class.forName().newInstance() does not work if the underlying + //InputSplit has package visibility + Constructor + constructor = + splitClass.getDeclaredConstructor(new Class[]{}); + constructor.setAccessible(true); + + split = constructor.newInstance(); + // read baseSplit from input + ((Writable) split).readFields(input); + this.baseMapRedSplit = split; + } catch (Exception e) { + throw new IOException("Exception from " + baseSplitClassName, e); + } + + String tableSchemaString = WritableUtils.readString(input); + tableSchema = (HCatSchema) HCatUtil.deserialize(tableSchemaString); + } + + /* (non-Javadoc) + * @see org.apache.hadoop.io.Writable#write(java.io.DataOutput) + */ + @Override + public void write(DataOutput output) throws IOException { + String partitionInfoString = HCatUtil.serialize(partitionInfo); + + // write partitionInfo into output + WritableUtils.writeString(output, partitionInfoString); + + WritableUtils.writeString(output, baseMapRedSplit.getClass().getName()); + Writable baseSplitWritable = (Writable) baseMapRedSplit; + //write baseSplit into output + baseSplitWritable.write(output); + + //write the table schema into output + String tableSchemaString = HCatUtil.serialize(tableSchema); + WritableUtils.writeString(output, tableSchemaString); + } + +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/HCatStorageHandler.java hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/HCatStorageHandler.java new file mode 100644 index 0000000..05adb54 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/HCatStorageHandler.java @@ -0,0 +1,121 @@ +/** + * 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.hcatalog.mapreduce; + +import java.util.Map; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.metadata.HiveStorageHandler; +import org.apache.hadoop.hive.ql.plan.TableDesc; +import org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider; +import org.apache.hadoop.mapred.OutputFormat; + +/** + * The abstract Class HCatStorageHandler would server as the base class for all + * the storage handlers required for non-native tables in HCatalog. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.HCatStorageHandler} instead + */ +public abstract class HCatStorageHandler implements HiveStorageHandler { + + //TODO move this to HiveStorageHandler + + /** + * This method is called to allow the StorageHandlers the chance + * to populate the JobContext.getConfiguration() with properties that + * maybe be needed by the handler's bundled artifacts (ie InputFormat, SerDe, etc). + * Key value pairs passed into jobProperties is guaranteed to be set in the job's + * configuration object. User's can retrieve "context" information from tableDesc. + * User's should avoid mutating tableDesc and only make changes in jobProperties. + * This method is expected to be idempotent such that a job called with the + * same tableDesc values should return the same key-value pairs in jobProperties. + * Any external state set by this method should remain the same if this method is + * called again. It is up to the user to determine how best guarantee this invariant. + * + * This method in particular is to create a configuration for input. + * @param tableDesc + * @param jobProperties + */ + public abstract void configureInputJobProperties(TableDesc tableDesc, Map jobProperties); + + //TODO move this to HiveStorageHandler + + /** + * This method is called to allow the StorageHandlers the chance + * to populate the JobContext.getConfiguration() with properties that + * maybe be needed by the handler's bundled artifacts (ie InputFormat, SerDe, etc). + * Key value pairs passed into jobProperties is guaranteed to be set in the job's + * configuration object. User's can retrieve "context" information from tableDesc. + * User's should avoid mutating tableDesc and only make changes in jobProperties. + * This method is expected to be idempotent such that a job called with the + * same tableDesc values should return the same key-value pairs in jobProperties. + * Any external state set by this method should remain the same if this method is + * called again. It is up to the user to determine how best guarantee this invariant. + * + * This method in particular is to create a configuration for output. + * @param tableDesc + * @param jobProperties + */ + public abstract void configureOutputJobProperties(TableDesc tableDesc, Map jobProperties); + + /** + * + * + * @return authorization provider + * @throws HiveException + */ + public abstract HiveAuthorizationProvider getAuthorizationProvider() + throws HiveException; + + /* + * (non-Javadoc) + * + * @see org.apache.hadoop.hive.ql.metadata.HiveStorageHandler# + * configureTableJobProperties(org.apache.hadoop.hive.ql.plan.TableDesc, + * java.util.Map) + */ + @Override + @Deprecated + public final void configureTableJobProperties(TableDesc tableDesc, + Map jobProperties) { + } + + /* + * (non-Javadoc) + * + * @see org.apache.hadoop.conf.Configurable#getConf() + */ + @Override + public abstract Configuration getConf(); + + /* + * (non-Javadoc) + * + * @see org.apache.hadoop.conf.Configurable#setConf(org.apache.hadoop.conf. + * Configuration) + */ + @Override + public abstract void setConf(Configuration conf); + + OutputFormatContainer getOutputFormatContainer(OutputFormat outputFormat) { + return new DefaultOutputFormatContainer(outputFormat); + } + +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/HCatTableInfo.java hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/HCatTableInfo.java new file mode 100644 index 0000000..7484b0b --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/HCatTableInfo.java @@ -0,0 +1,188 @@ +/** + * 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.hcatalog.mapreduce; + +import java.io.IOException; +import java.io.Serializable; + +import org.apache.hadoop.hive.metastore.MetaStoreUtils; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hcatalog.common.HCatUtil; +import org.apache.hcatalog.data.schema.HCatSchema; + +/** + * + * HCatTableInfo - class to communicate table information to {@link HCatInputFormat} + * and {@link HCatOutputFormat} + * + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.HCatTableInfo} instead + */ +public class HCatTableInfo implements Serializable { + + + private static final long serialVersionUID = 1L; + + /** The db and table names */ + private final String databaseName; + private final String tableName; + + /** The table schema. */ + private final HCatSchema dataColumns; + private final HCatSchema partitionColumns; + + /** The table being written to */ + private final Table table; + + /** The storer info */ + private StorerInfo storerInfo; + + /** + * Initializes a new HCatTableInfo instance to be used with {@link HCatInputFormat} + * for reading data from a table. + * work with hadoop security, the kerberos principal name of the server - else null + * The principal name should be of the form: + * /_HOST@ like "hcat/_HOST@myrealm.com" + * The special string _HOST will be replaced automatically with the correct host name + * @param databaseName the db name + * @param tableName the table name + * @param dataColumns schema of columns which contain data + * @param partitionColumns schema of partition columns + * @param storerInfo information about storage descriptor + * @param table hive metastore table class + */ + HCatTableInfo( + String databaseName, + String tableName, + HCatSchema dataColumns, + HCatSchema partitionColumns, + StorerInfo storerInfo, + Table table) { + this.databaseName = (databaseName == null) ? MetaStoreUtils.DEFAULT_DATABASE_NAME : databaseName; + this.tableName = tableName; + this.dataColumns = dataColumns; + this.table = table; + this.storerInfo = storerInfo; + this.partitionColumns = partitionColumns; + } + + /** + * Gets the value of databaseName + * @return the databaseName + */ + public String getDatabaseName() { + return databaseName; + } + + /** + * Gets the value of tableName + * @return the tableName + */ + public String getTableName() { + return tableName; + } + + /** + * @return return schema of data columns as defined in meta store + */ + public HCatSchema getDataColumns() { + return dataColumns; + } + + /** + * @return schema of partition columns + */ + public HCatSchema getPartitionColumns() { + return partitionColumns; + } + + /** + * @return the storerInfo + */ + public StorerInfo getStorerInfo() { + return storerInfo; + } + + public String getTableLocation() { + return table.getSd().getLocation(); + } + + /** + * minimize dependency on hive classes so this is package private + * this should eventually no longer be used + * @return hive metastore representation of table + */ + Table getTable() { + return table; + } + + /** + * create an HCatTableInfo instance from the supplied Hive Table instance + * @param table to create an instance from + * @return HCatTableInfo + * @throws IOException + */ + static HCatTableInfo valueOf(Table table) throws IOException { + // Explicitly use {@link org.apache.hadoop.hive.ql.metadata.Table} when getting the schema, + // but store @{link org.apache.hadoop.hive.metastore.api.Table} as this class is serialized + // into the job conf. + org.apache.hadoop.hive.ql.metadata.Table mTable = + new org.apache.hadoop.hive.ql.metadata.Table(table); + HCatSchema schema = HCatUtil.extractSchema(mTable); + StorerInfo storerInfo = + InternalUtil.extractStorerInfo(table.getSd(), table.getParameters()); + HCatSchema partitionColumns = HCatUtil.getPartitionColumns(mTable); + return new HCatTableInfo(table.getDbName(), table.getTableName(), schema, + partitionColumns, storerInfo, table); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + HCatTableInfo tableInfo = (HCatTableInfo) o; + + if (dataColumns != null ? !dataColumns.equals(tableInfo.dataColumns) : tableInfo.dataColumns != null) + return false; + if (databaseName != null ? !databaseName.equals(tableInfo.databaseName) : tableInfo.databaseName != null) + return false; + if (partitionColumns != null ? !partitionColumns.equals(tableInfo.partitionColumns) : tableInfo.partitionColumns != null) + return false; + if (storerInfo != null ? !storerInfo.equals(tableInfo.storerInfo) : tableInfo.storerInfo != null) return false; + if (table != null ? !table.equals(tableInfo.table) : tableInfo.table != null) return false; + if (tableName != null ? !tableName.equals(tableInfo.tableName) : tableInfo.tableName != null) return false; + + return true; + } + + + @Override + public int hashCode() { + int result = databaseName != null ? databaseName.hashCode() : 0; + result = 31 * result + (tableName != null ? tableName.hashCode() : 0); + result = 31 * result + (dataColumns != null ? dataColumns.hashCode() : 0); + result = 31 * result + (partitionColumns != null ? partitionColumns.hashCode() : 0); + result = 31 * result + (table != null ? table.hashCode() : 0); + result = 31 * result + (storerInfo != null ? storerInfo.hashCode() : 0); + return result; + } + +} + diff --git hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/InitializeInput.java hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/InitializeInput.java new file mode 100644 index 0000000..5f6bc9b --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/InitializeInput.java @@ -0,0 +1,172 @@ +/** + * 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.hcatalog.mapreduce; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hcatalog.common.ErrorType; +import org.apache.hcatalog.common.HCatConstants; +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.common.HCatUtil; +import org.apache.hcatalog.data.schema.HCatSchema; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The Class which handles querying the metadata server using the MetaStoreClient. The list of + * partitions matching the partition filter is fetched from the server and the information is + * serialized and written into the JobContext configuration. The inputInfo is also updated with + * info required in the client process context. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.InitializeInput} instead + */ +class InitializeInput { + + private static final Logger LOG = LoggerFactory.getLogger(InitializeInput.class); + + /** + * @see org.apache.hcatalog.mapreduce.InitializeInput#setInput(org.apache.hadoop.conf.Configuration, InputJobInfo) + */ + public static void setInput(Job job, InputJobInfo theirInputJobInfo) throws Exception { + setInput(job.getConfiguration(), theirInputJobInfo); + } + + /** + * Set the input to use for the Job. This queries the metadata server with the specified + * partition predicates, gets the matching partitions, and puts the information in the job + * configuration object. + * + * To ensure a known InputJobInfo state, only the database name, table name, filter, and + * properties are preserved. All other modification from the given InputJobInfo are discarded. + * + * After calling setInput, InputJobInfo can be retrieved from the job configuration as follows: + * {code} + * InputJobInfo inputInfo = (InputJobInfo) HCatUtil.deserialize( + * job.getConfiguration().get(HCatConstants.HCAT_KEY_JOB_INFO)); + * {code} + * + * @param conf the job Configuration object + * @param theirInputJobInfo information on the Input to read + * @throws Exception + */ + public static void setInput(Configuration conf, + InputJobInfo theirInputJobInfo) throws Exception { + InputJobInfo inputJobInfo = InputJobInfo.create( + theirInputJobInfo.getDatabaseName(), + theirInputJobInfo.getTableName(), + theirInputJobInfo.getFilter(), + theirInputJobInfo.getProperties()); + conf.set( + HCatConstants.HCAT_KEY_JOB_INFO, + HCatUtil.serialize(getInputJobInfo(conf, inputJobInfo, null))); + } + + /** + * Returns the given InputJobInfo after populating with data queried from the metadata service. + */ + private static InputJobInfo getInputJobInfo( + Configuration conf, InputJobInfo inputJobInfo, String locationFilter) throws Exception { + HiveMetaStoreClient client = null; + HiveConf hiveConf = null; + try { + if (conf != null) { + hiveConf = HCatUtil.getHiveConf(conf); + } else { + hiveConf = new HiveConf(HCatInputFormat.class); + } + client = HCatUtil.getHiveClient(hiveConf); + Table table = HCatUtil.getTable(client, inputJobInfo.getDatabaseName(), + inputJobInfo.getTableName()); + + List partInfoList = new ArrayList(); + + inputJobInfo.setTableInfo(HCatTableInfo.valueOf(table.getTTable())); + if (table.getPartitionKeys().size() != 0) { + //Partitioned table + List parts = client.listPartitionsByFilter(inputJobInfo.getDatabaseName(), + inputJobInfo.getTableName(), + inputJobInfo.getFilter(), + (short) -1); + + // Default to 100,000 partitions if hive.metastore.maxpartition is not defined + int maxPart = hiveConf.getInt("hcat.metastore.maxpartitions", 100000); + if (parts != null && parts.size() > maxPart) { + throw new HCatException(ErrorType.ERROR_EXCEED_MAXPART, "total number of partitions is " + parts.size()); + } + + // populate partition info + for (Partition ptn : parts) { + HCatSchema schema = HCatUtil.extractSchema( + new org.apache.hadoop.hive.ql.metadata.Partition(table, ptn)); + PartInfo partInfo = extractPartInfo(schema, ptn.getSd(), + ptn.getParameters(), conf, inputJobInfo); + partInfo.setPartitionValues(InternalUtil.createPtnKeyValueMap(table, ptn)); + partInfoList.add(partInfo); + } + + } else { + //Non partitioned table + HCatSchema schema = HCatUtil.extractSchema(table); + PartInfo partInfo = extractPartInfo(schema, table.getTTable().getSd(), + table.getParameters(), conf, inputJobInfo); + partInfo.setPartitionValues(new HashMap()); + partInfoList.add(partInfo); + } + inputJobInfo.setPartitions(partInfoList); + + return inputJobInfo; + } finally { + HCatUtil.closeHiveClientQuietly(client); + } + + } + + private static PartInfo extractPartInfo(HCatSchema schema, StorageDescriptor sd, + Map parameters, Configuration conf, + InputJobInfo inputJobInfo) throws IOException { + + StorerInfo storerInfo = InternalUtil.extractStorerInfo(sd, parameters); + + Properties hcatProperties = new Properties(); + HCatStorageHandler storageHandler = HCatUtil.getStorageHandler(conf, storerInfo); + + // copy the properties from storageHandler to jobProperties + Map jobProperties = HCatUtil.getInputJobProperties(storageHandler, inputJobInfo); + + for (String key : parameters.keySet()) { + hcatProperties.put(key, parameters.get(key)); + } + // FIXME + // Bloating partinfo with inputJobInfo is not good + return new PartInfo(schema, storageHandler, sd.getLocation(), + hcatProperties, jobProperties, inputJobInfo.getTableInfo()); + } + +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/InputJobInfo.java hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/InputJobInfo.java new file mode 100644 index 0000000..ed8a501 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/InputJobInfo.java @@ -0,0 +1,202 @@ +/** + * 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.hcatalog.mapreduce; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hive.metastore.MetaStoreUtils; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import java.util.List; +import java.util.Properties; +import java.util.zip.Deflater; +import java.util.zip.DeflaterOutputStream; +import java.util.zip.InflaterInputStream; + +/** + * Container for metadata read from the metadata server. + * Prior to release 0.5, InputJobInfo was a key part of the public API, exposed directly + * to end-users as an argument to + * {@link HCatInputFormat#setInput(org.apache.hadoop.mapreduce.Job, InputJobInfo)}. + * Going forward, we plan on treating InputJobInfo as an implementation detail and no longer + * expose to end-users. Should you have a need to use InputJobInfo outside HCatalog itself, + * please contact the developer mailing list before depending on this class. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.InputJobInfo} instead + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public class InputJobInfo implements Serializable { + + /** The serialization version */ + private static final long serialVersionUID = 1L; + + /** The db and table names. */ + private final String databaseName; + private final String tableName; + + /** meta information of the table to be read from */ + private HCatTableInfo tableInfo; + + /** The partition filter */ + private String filter; + + /** The list of partitions matching the filter. */ + transient private List partitions; + + /** implementation specific job properties */ + private Properties properties; + + /** + * Initializes a new InputJobInfo + * for reading data from a table. + * @param databaseName the db name + * @param tableName the table name + * @param filter the partition filter + * @param properties implementation specific job properties + */ + public static InputJobInfo create(String databaseName, + String tableName, + String filter, + Properties properties) { + return new InputJobInfo(databaseName, tableName, filter, properties); + } + + /** + * Initializes a new InputJobInfo + * for reading data from a table. + * @param databaseName the db name + * @param tableName the table name + * @param filter the partition filter + */ + @Deprecated + public static InputJobInfo create(String databaseName, + String tableName, + String filter) { + return create(databaseName, tableName, filter, null); + } + + + private InputJobInfo(String databaseName, + String tableName, + String filter, + Properties properties) { + this.databaseName = (databaseName == null) ? + MetaStoreUtils.DEFAULT_DATABASE_NAME : databaseName; + this.tableName = tableName; + this.filter = filter; + this.properties = properties == null ? new Properties() : properties; + } + + /** + * Gets the value of databaseName + * @return the databaseName + */ + public String getDatabaseName() { + return databaseName; + } + + /** + * Gets the value of tableName + * @return the tableName + */ + public String getTableName() { + return tableName; + } + + /** + * Gets the table's meta information + * @return the HCatTableInfo + */ + public HCatTableInfo getTableInfo() { + return tableInfo; + } + + /** + * set the tablInfo instance + * this should be the same instance + * determined by this object's DatabaseName and TableName + * @param tableInfo + */ + void setTableInfo(HCatTableInfo tableInfo) { + this.tableInfo = tableInfo; + } + + /** + * Gets the value of partition filter + * @return the filter string + */ + public String getFilter() { + return filter; + } + + /** + * @return partition info + */ + public List getPartitions() { + return partitions; + } + + /** + * @return partition info list + */ + void setPartitions(List partitions) { + this.partitions = partitions; + } + + /** + * Set/Get Property information to be passed down to *StorageHandler implementation + * put implementation specific storage handler configurations here + * @return the implementation specific job properties + */ + public Properties getProperties() { + return properties; + } + + /** + * Serialize this object, compressing the partitions which can exceed the + * allowed jobConf size. + * @see HCATALOG-453 + */ + private void writeObject(ObjectOutputStream oos) + throws IOException { + oos.defaultWriteObject(); + Deflater def = new Deflater(Deflater.BEST_COMPRESSION); + ObjectOutputStream partInfoWriter = + new ObjectOutputStream(new DeflaterOutputStream(oos, def)); + partInfoWriter.writeObject(partitions); + partInfoWriter.close(); + } + + /** + * Deserialize this object, decompressing the partitions which can exceed the + * allowed jobConf size. + * @see HCATALOG-453 + */ + @SuppressWarnings("unchecked") + private void readObject(ObjectInputStream ois) + throws IOException, ClassNotFoundException { + ois.defaultReadObject(); + ObjectInputStream partInfoReader = + new ObjectInputStream(new InflaterInputStream(ois)); + partitions = (List)partInfoReader.readObject(); + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/InternalUtil.java hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/InternalUtil.java new file mode 100644 index 0000000..4978b21 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/InternalUtil.java @@ -0,0 +1,221 @@ +/** + * 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.hcatalog.mapreduce; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.metastore.MetaStoreUtils; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.serde2.Deserializer; +import org.apache.hadoop.hive.serde2.SerDe; +import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.MapObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; +import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; +import org.apache.hadoop.mapred.Reporter; +import org.apache.hadoop.mapreduce.InputSplit; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hcatalog.common.HCatUtil; +import org.apache.hcatalog.data.schema.HCatFieldSchema; +import org.apache.hcatalog.data.schema.HCatSchema; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.InternalUtil} instead + */ +class InternalUtil { + private static final Logger LOG = LoggerFactory.getLogger(InternalUtil.class); + + static StorerInfo extractStorerInfo(StorageDescriptor sd, Map properties) throws IOException { + Properties hcatProperties = new Properties(); + for (String key : properties.keySet()) { + hcatProperties.put(key, properties.get(key)); + } + + // also populate with StorageDescriptor->SerDe.Parameters + for (Map.Entry param : + sd.getSerdeInfo().getParameters().entrySet()) { + hcatProperties.put(param.getKey(), param.getValue()); + } + + + return new StorerInfo( + sd.getInputFormat(), sd.getOutputFormat(), sd.getSerdeInfo().getSerializationLib(), + properties.get(org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_STORAGE), + hcatProperties); + } + + static StructObjectInspector createStructObjectInspector(HCatSchema outputSchema) throws IOException { + + if (outputSchema == null) { + throw new IOException("Invalid output schema specified"); + } + + List fieldInspectors = new ArrayList(); + List fieldNames = new ArrayList(); + + for (HCatFieldSchema hcatFieldSchema : outputSchema.getFields()) { + TypeInfo type = TypeInfoUtils.getTypeInfoFromTypeString(hcatFieldSchema.getTypeString()); + + fieldNames.add(hcatFieldSchema.getName()); + fieldInspectors.add(getObjectInspector(type)); + } + + StructObjectInspector structInspector = ObjectInspectorFactory. + getStandardStructObjectInspector(fieldNames, fieldInspectors); + return structInspector; + } + + private static ObjectInspector getObjectInspector(TypeInfo type) throws IOException { + + switch (type.getCategory()) { + + case PRIMITIVE: + PrimitiveTypeInfo primitiveType = (PrimitiveTypeInfo) type; + return PrimitiveObjectInspectorFactory. + getPrimitiveJavaObjectInspector(primitiveType.getPrimitiveCategory()); + + case MAP: + MapTypeInfo mapType = (MapTypeInfo) type; + MapObjectInspector mapInspector = ObjectInspectorFactory.getStandardMapObjectInspector( + getObjectInspector(mapType.getMapKeyTypeInfo()), getObjectInspector(mapType.getMapValueTypeInfo())); + return mapInspector; + + case LIST: + ListTypeInfo listType = (ListTypeInfo) type; + ListObjectInspector listInspector = ObjectInspectorFactory.getStandardListObjectInspector( + getObjectInspector(listType.getListElementTypeInfo())); + return listInspector; + + case STRUCT: + StructTypeInfo structType = (StructTypeInfo) type; + List fieldTypes = structType.getAllStructFieldTypeInfos(); + + List fieldInspectors = new ArrayList(); + for (TypeInfo fieldType : fieldTypes) { + fieldInspectors.add(getObjectInspector(fieldType)); + } + + StructObjectInspector structInspector = ObjectInspectorFactory.getStandardStructObjectInspector( + structType.getAllStructFieldNames(), fieldInspectors); + return structInspector; + + default: + throw new IOException("Unknown field schema type"); + } + } + + //TODO this has to find a better home, it's also hardcoded as default in hive would be nice + // if the default was decided by the serde + static void initializeOutputSerDe(SerDe serDe, Configuration conf, OutputJobInfo jobInfo) + throws SerDeException { + serDe.initialize(conf, getSerdeProperties(jobInfo.getTableInfo(), jobInfo.getOutputSchema())); + } + + static void initializeDeserializer(Deserializer deserializer, Configuration conf, + HCatTableInfo info, HCatSchema schema) throws SerDeException { + Properties props = getSerdeProperties(info, schema); + LOG.info("Initializing " + deserializer.getClass().getName() + " with properties " + props); + deserializer.initialize(conf, props); + } + + private static Properties getSerdeProperties(HCatTableInfo info, HCatSchema s) + throws SerDeException { + Properties props = new Properties(); + List fields = HCatUtil.getFieldSchemaList(s.getFields()); + props.setProperty(org.apache.hadoop.hive.serde.serdeConstants.LIST_COLUMNS, + MetaStoreUtils.getColumnNamesFromFieldSchema(fields)); + props.setProperty(org.apache.hadoop.hive.serde.serdeConstants.LIST_COLUMN_TYPES, + MetaStoreUtils.getColumnTypesFromFieldSchema(fields)); + + // setting these props to match LazySimpleSerde + props.setProperty(org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_NULL_FORMAT, "\\N"); + props.setProperty(org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_FORMAT, "1"); + + //add props from params set in table schema + props.putAll(info.getStorerInfo().getProperties()); + + return props; + } + + static Reporter createReporter(TaskAttemptContext context) { + return new ProgressReporter(context); + } + + /** + * Casts an InputSplit into a HCatSplit, providing a useful error message if the cast fails. + * @param split the InputSplit + * @return the HCatSplit + * @throws IOException + */ + public static HCatSplit castToHCatSplit(InputSplit split) throws IOException { + if (split instanceof HCatSplit) { + return (HCatSplit) split; + } else { + throw new IOException("Split must be " + HCatSplit.class.getName() + + " but found " + split.getClass().getName()); + } + } + + + static Map createPtnKeyValueMap(Table table, Partition ptn) + throws IOException { + List values = ptn.getValues(); + if (values.size() != table.getPartitionKeys().size()) { + throw new IOException( + "Partition values in partition inconsistent with table definition, table " + + table.getTableName() + " has " + + table.getPartitionKeys().size() + + " partition keys, partition has " + values.size() + + "partition values"); + } + + Map ptnKeyValues = new HashMap(); + + int i = 0; + for (FieldSchema schema : table.getPartitionKeys()) { + // CONCERN : the way this mapping goes, the order *needs* to be + // preserved for table.getPartitionKeys() and ptn.getValues() + ptnKeyValues.put(schema.getName().toLowerCase(), values.get(i)); + i++; + } + + return ptnKeyValues; + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/MultiOutputFormat.java hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/MultiOutputFormat.java new file mode 100644 index 0000000..d6da8dd --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/MultiOutputFormat.java @@ -0,0 +1,620 @@ +/** + * 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.hcatalog.mapreduce; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; + +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.shims.HadoopShims; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.JobStatus.State; +import org.apache.hadoop.mapreduce.OutputCommitter; +import org.apache.hadoop.mapreduce.OutputFormat; +import org.apache.hadoop.mapreduce.RecordWriter; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.TaskInputOutputContext; +import org.apache.hadoop.util.ReflectionUtils; +import org.apache.hcatalog.common.HCatUtil; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The MultiOutputFormat class simplifies writing output data to multiple + * outputs. + *

+ * Multiple output formats can be defined each with its own + * OutputFormat class, own key class and own value class. Any + * configuration on these output format classes can be done without interfering + * with other output format's configuration. + *

+ * Usage pattern for job submission: + * + *

+ *
+ * Job job = new Job();
+ *
+ * FileInputFormat.setInputPath(job, inDir);
+ *
+ * job.setMapperClass(WordCountMap.class);
+ * job.setReducerClass(WordCountReduce.class);
+ * job.setInputFormatClass(TextInputFormat.class);
+ * job.setOutputFormatClass(MultiOutputFormat.class);
+ * // Need not define OutputKeyClass and OutputValueClass. They default to
+ * // Writable.class
+ * job.setMapOutputKeyClass(Text.class);
+ * job.setMapOutputValueClass(IntWritable.class);
+ *
+ *
+ * // Create a JobConfigurer that will configure the job with the multiple
+ * // output format information.
+ * JobConfigurer configurer = MultiOutputFormat.createConfigurer(job);
+ *
+ * // Defines additional single text based output 'text' for the job.
+ * // Any configuration for the defined OutputFormat should be done with
+ * // the Job obtained with configurer.getJob() method.
+ * configurer.addOutputFormat("text", TextOutputFormat.class,
+ *                 IntWritable.class, Text.class);
+ * FileOutputFormat.setOutputPath(configurer.getJob("text"), textOutDir);
+ *
+ * // Defines additional sequence-file based output 'sequence' for the job
+ * configurer.addOutputFormat("sequence", SequenceFileOutputFormat.class,
+ *                 Text.class, IntWritable.class);
+ * FileOutputFormat.setOutputPath(configurer.getJob("sequence"), seqOutDir);
+ * ...
+ * // configure method to be called on the JobConfigurer once all the
+ * // output formats have been defined and configured.
+ * configurer.configure();
+ *
+ * job.waitForCompletion(true);
+ * ...
+ * 
+ *

+ * Usage in Reducer: + * + *

+ * public class WordCountReduce extends
+ *         Reducer<Text, IntWritable, Writable, Writable> {
+ *
+ *     private IntWritable count = new IntWritable();
+ *
+ *     public void reduce(Text word, Iterator<IntWritable> values,
+ *             Context context)
+ *             throws IOException {
+ *         int sum = 0;
+ *         for (IntWritable val : values) {
+ *             sum += val.get();
+ *         }
+ *         count.set(sum);
+ *         MultiOutputFormat.write("text", count, word, context);
+ *         MultiOutputFormat.write("sequence", word, count, context);
+ *     }
+ *
+ * }
+ *
+ * 
+ * + * Map only jobs: + *

+ * MultiOutputFormat.write("output", key, value, context); can be called similar + * to a reducer in map only jobs. + * + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.MultiOutputFormat} instead + */ +public class MultiOutputFormat extends OutputFormat { + + private static final Logger LOGGER = LoggerFactory.getLogger(MultiOutputFormat.class.getName()); + private static final String MO_ALIASES = "mapreduce.multiout.aliases"; + private static final String MO_ALIAS = "mapreduce.multiout.alias"; + private static final String CONF_KEY_DELIM = "%%"; + private static final String CONF_VALUE_DELIM = ";;"; + private static final String COMMA_DELIM = ","; + private static final List configsToOverride = new ArrayList(); + private static final Map configsToMerge = new HashMap(); + + static { + configsToOverride.add("mapred.output.dir"); + configsToOverride.add(ShimLoader.getHadoopShims().getHCatShim().getPropertyName(HadoopShims.HCatHadoopShims.PropertyName.CACHE_SYMLINK)); + configsToMerge.put(JobContext.JOB_NAMENODES, COMMA_DELIM); + configsToMerge.put("tmpfiles", COMMA_DELIM); + configsToMerge.put("tmpjars", COMMA_DELIM); + configsToMerge.put("tmparchives", COMMA_DELIM); + configsToMerge.put(ShimLoader.getHadoopShims().getHCatShim().getPropertyName(HadoopShims.HCatHadoopShims.PropertyName.CACHE_ARCHIVES), COMMA_DELIM); + configsToMerge.put(ShimLoader.getHadoopShims().getHCatShim().getPropertyName(HadoopShims.HCatHadoopShims.PropertyName.CACHE_FILES), COMMA_DELIM); + String fileSep; + if (HCatUtil.isHadoop23()) { + fileSep = ","; + } else { + fileSep = System.getProperty("path.separator"); + } + configsToMerge.put("mapred.job.classpath.archives", fileSep); + configsToMerge.put("mapred.job.classpath.files", fileSep); + } + + /** + * Get a JobConfigurer instance that will support configuration of the job + * for multiple output formats. + * + * @param job the mapreduce job to be submitted + * @return JobConfigurer + */ + public static JobConfigurer createConfigurer(Job job) { + return JobConfigurer.create(job); + } + + /** + * Get the JobContext with the related OutputFormat configuration populated given the alias + * and the actual JobContext + * @param alias the name given to the OutputFormat configuration + * @param context the JobContext + * @return a copy of the JobContext with the alias configuration populated + */ + public static JobContext getJobContext(String alias, JobContext context) { + String aliasConf = context.getConfiguration().get(getAliasConfName(alias)); + JobContext aliasContext = ShimLoader.getHadoopShims().getHCatShim().createJobContext(context.getConfiguration(), context.getJobID()); + addToConfig(aliasConf, aliasContext.getConfiguration()); + return aliasContext; + } + + /** + * Get the TaskAttemptContext with the related OutputFormat configuration populated given the alias + * and the actual TaskAttemptContext + * @param alias the name given to the OutputFormat configuration + * @param context the Mapper or Reducer Context + * @return a copy of the TaskAttemptContext with the alias configuration populated + */ + public static TaskAttemptContext getTaskAttemptContext(String alias, TaskAttemptContext context) { + String aliasConf = context.getConfiguration().get(getAliasConfName(alias)); + TaskAttemptContext aliasContext = ShimLoader.getHadoopShims().getHCatShim().createTaskAttemptContext( + context.getConfiguration(), context.getTaskAttemptID()); + addToConfig(aliasConf, aliasContext.getConfiguration()); + return aliasContext; + } + + /** + * Write the output key and value using the OutputFormat defined by the + * alias. + * + * @param alias the name given to the OutputFormat configuration + * @param key the output key to be written + * @param value the output value to be written + * @param context the Mapper or Reducer Context + * @throws IOException + * @throws InterruptedException + */ + public static void write(String alias, K key, V value, TaskInputOutputContext context) + throws IOException, InterruptedException { + KeyValue keyval = new KeyValue(key, value); + context.write(new Text(alias), keyval); + } + + @Override + public void checkOutputSpecs(JobContext context) throws IOException, InterruptedException { + for (String alias : getOutputFormatAliases(context)) { + LOGGER.debug("Calling checkOutputSpecs for alias: " + alias); + JobContext aliasContext = getJobContext(alias, context); + OutputFormat outputFormat = getOutputFormatInstance(aliasContext); + outputFormat.checkOutputSpecs(aliasContext); + // Copy credentials and any new config added back to JobContext + context.getCredentials().addAll(aliasContext.getCredentials()); + setAliasConf(alias, context, aliasContext); + } + } + + @Override + public RecordWriter getRecordWriter(TaskAttemptContext context) + throws IOException, + InterruptedException { + return new MultiRecordWriter(context); + } + + @Override + public OutputCommitter getOutputCommitter(TaskAttemptContext context) throws IOException, + InterruptedException { + return new MultiOutputCommitter(context); + } + + private static OutputFormat getOutputFormatInstance(JobContext context) { + OutputFormat outputFormat; + try { + outputFormat = ReflectionUtils.newInstance(context.getOutputFormatClass(), + context.getConfiguration()); + } catch (ClassNotFoundException e) { + throw new IllegalStateException(e); + } + return outputFormat; + } + + private static String[] getOutputFormatAliases(JobContext context) { + return context.getConfiguration().getStrings(MO_ALIASES); + } + + /** + * Compare the aliasContext with userJob and add the differing configuration + * as mapreduce.multiout.alias..conf to the userJob. + *

+ * Merge config like tmpjars, tmpfile, tmparchives, + * mapreduce.job.hdfs-servers that are directly handled by JobClient and add + * them to userJob. + *

+ * Add mapred.output.dir config to userJob. + * + * @param alias alias name associated with a OutputFormat + * @param userJob reference to Job that the user is going to submit + * @param aliasContext JobContext populated with OutputFormat related + * configuration. + */ + private static void setAliasConf(String alias, JobContext userJob, JobContext aliasContext) { + Configuration userConf = userJob.getConfiguration(); + StringBuilder builder = new StringBuilder(); + for (Entry conf : aliasContext.getConfiguration()) { + String key = conf.getKey(); + String value = conf.getValue(); + String jobValue = userConf.getRaw(key); + if (jobValue == null || !jobValue.equals(value)) { + if (configsToMerge.containsKey(key)) { + String mergedValue = getMergedConfValue(jobValue, value, configsToMerge.get(key)); + userConf.set(key, mergedValue); + } else { + if (configsToOverride.contains(key)) { + userConf.set(key, value); + } + builder.append(key).append(CONF_KEY_DELIM).append(value) + .append(CONF_VALUE_DELIM); + } + } + } + if (builder.length() > CONF_VALUE_DELIM.length()) { + builder.delete(builder.length() - CONF_VALUE_DELIM.length(), builder.length()); + userConf.set(getAliasConfName(alias), builder.toString()); + } + } + + private static String getMergedConfValue(String originalValues, String newValues, String separator) { + if (originalValues == null) { + return newValues; + } + Set mergedValues = new LinkedHashSet(); + mergedValues.addAll(Arrays.asList(StringUtils.split(originalValues, separator))); + mergedValues.addAll(Arrays.asList(StringUtils.split(newValues, separator))); + StringBuilder builder = new StringBuilder(originalValues.length() + newValues.length() + 2); + for (String value : mergedValues) { + builder.append(value).append(separator); + } + return builder.substring(0, builder.length() - separator.length()); + } + + private static String getAliasConfName(String alias) { + return MO_ALIAS + "." + alias + ".conf"; + } + + private static void addToConfig(String aliasConf, Configuration conf) { + String[] config = aliasConf.split(CONF_KEY_DELIM + "|" + CONF_VALUE_DELIM); + for (int i = 0; i < config.length; i += 2) { + conf.set(config[i], config[i + 1]); + } + } + + /** + * Class that supports configuration of the job for multiple output formats. + */ + public static class JobConfigurer { + + private final Job job; + private Map outputConfigs = new LinkedHashMap(); + + private JobConfigurer(Job job) { + this.job = job; + } + + private static JobConfigurer create(Job job) { + JobConfigurer configurer = new JobConfigurer(job); + return configurer; + } + + /** + * Add a OutputFormat configuration to the Job with a alias name. + * + * @param alias the name to be given to the OutputFormat configuration + * @param outputFormatClass OutputFormat class + * @param keyClass the key class for the output data + * @param valueClass the value class for the output data + * @throws IOException + */ + public void addOutputFormat(String alias, + Class outputFormatClass, + Class keyClass, Class valueClass) throws IOException { + Job copy = new Job(this.job.getConfiguration()); + outputConfigs.put(alias, copy); + copy.setOutputFormatClass(outputFormatClass); + copy.setOutputKeyClass(keyClass); + copy.setOutputValueClass(valueClass); + } + + /** + * Get the Job configuration for a OutputFormat defined by the alias + * name. The job returned by this method should be passed to the + * OutputFormat for any configuration instead of the Job that will be + * submitted to the JobClient. + * + * @param alias the name used for the OutputFormat during + * addOutputFormat + * @return Job + */ + public Job getJob(String alias) { + Job copy = outputConfigs.get(alias); + if (copy == null) { + throw new IllegalArgumentException("OutputFormat with alias " + alias + + " has not beed added"); + } + return copy; + } + + /** + * Configure the job with the multiple output formats added. This method + * should be called after all the output formats have been added and + * configured and before the job submission. + */ + public void configure() { + StringBuilder aliases = new StringBuilder(); + Configuration jobConf = job.getConfiguration(); + for (Entry entry : outputConfigs.entrySet()) { + // Copy credentials + job.getCredentials().addAll(entry.getValue().getCredentials()); + String alias = entry.getKey(); + aliases.append(alias).append(COMMA_DELIM); + // Store the differing configuration for each alias in the job + // as a setting. + setAliasConf(alias, job, entry.getValue()); + } + aliases.delete(aliases.length() - COMMA_DELIM.length(), aliases.length()); + jobConf.set(MO_ALIASES, aliases.toString()); + } + + } + + private static class KeyValue implements Writable { + private final K key; + private final V value; + + public KeyValue(K key, V value) { + this.key = key; + this.value = value; + } + + public K getKey() { + return key; + } + + public V getValue() { + return value; + } + + @Override + public void write(DataOutput out) throws IOException { + // Ignore. Not required as this will be never + // serialized/deserialized. + } + + @Override + public void readFields(DataInput in) throws IOException { + // Ignore. Not required as this will be never + // serialized/deserialized. + } + } + + private static class MultiRecordWriter extends RecordWriter { + + private final Map baseRecordWriters; + + public MultiRecordWriter(TaskAttemptContext context) throws IOException, + InterruptedException { + baseRecordWriters = new LinkedHashMap(); + String[] aliases = getOutputFormatAliases(context); + for (String alias : aliases) { + LOGGER.info("Creating record writer for alias: " + alias); + TaskAttemptContext aliasContext = getTaskAttemptContext(alias, context); + Configuration aliasConf = aliasContext.getConfiguration(); + // Create output directory if not already created. + String outDir = aliasConf.get("mapred.output.dir"); + if (outDir != null) { + Path outputDir = new Path(outDir); + FileSystem fs = outputDir.getFileSystem(aliasConf); + if (!fs.exists(outputDir)) { + fs.mkdirs(outputDir); + } + } + OutputFormat outputFormat = getOutputFormatInstance(aliasContext); + baseRecordWriters.put(alias, + new BaseRecordWriterContainer(outputFormat.getRecordWriter(aliasContext), + aliasContext)); + } + } + + @Override + public void write(Writable key, Writable value) throws IOException, InterruptedException { + Text _key = (Text) key; + KeyValue _value = (KeyValue) value; + String alias = new String(_key.getBytes(), 0, _key.getLength()); + BaseRecordWriterContainer baseRWContainer = baseRecordWriters.get(alias); + if (baseRWContainer == null) { + throw new IllegalArgumentException("OutputFormat with alias " + alias + + " has not been added"); + } + baseRWContainer.getRecordWriter().write(_value.getKey(), _value.getValue()); + } + + @Override + public void close(TaskAttemptContext context) throws IOException, InterruptedException { + for (Entry entry : baseRecordWriters.entrySet()) { + BaseRecordWriterContainer baseRWContainer = entry.getValue(); + LOGGER.info("Closing record writer for alias: " + entry.getKey()); + baseRWContainer.getRecordWriter().close(baseRWContainer.getContext()); + } + } + + } + + private static class BaseRecordWriterContainer { + + private final RecordWriter recordWriter; + private final TaskAttemptContext context; + + public BaseRecordWriterContainer(RecordWriter recordWriter, TaskAttemptContext context) { + this.recordWriter = recordWriter; + this.context = context; + } + + public RecordWriter getRecordWriter() { + return recordWriter; + } + + public TaskAttemptContext getContext() { + return context; + } + } + + public class MultiOutputCommitter extends OutputCommitter { + + private final Map outputCommitters; + + public MultiOutputCommitter(TaskAttemptContext context) throws IOException, + InterruptedException { + outputCommitters = new LinkedHashMap(); + String[] aliases = getOutputFormatAliases(context); + for (String alias : aliases) { + LOGGER.info("Creating output committer for alias: " + alias); + TaskAttemptContext aliasContext = getTaskAttemptContext(alias, context); + OutputCommitter baseCommitter = getOutputFormatInstance(aliasContext) + .getOutputCommitter(aliasContext); + outputCommitters.put(alias, + new BaseOutputCommitterContainer(baseCommitter, aliasContext)); + } + } + + @Override + public void setupJob(JobContext jobContext) throws IOException { + for (String alias : outputCommitters.keySet()) { + LOGGER.info("Calling setupJob for alias: " + alias); + BaseOutputCommitterContainer outputContainer = outputCommitters.get(alias); + outputContainer.getBaseCommitter().setupJob(outputContainer.getContext()); + } + } + + @Override + public void setupTask(TaskAttemptContext taskContext) throws IOException { + for (String alias : outputCommitters.keySet()) { + LOGGER.info("Calling setupTask for alias: " + alias); + BaseOutputCommitterContainer outputContainer = outputCommitters.get(alias); + outputContainer.getBaseCommitter().setupTask(outputContainer.getContext()); + } + } + + @Override + public boolean needsTaskCommit(TaskAttemptContext taskContext) throws IOException { + boolean needTaskCommit = false; + for (String alias : outputCommitters.keySet()) { + BaseOutputCommitterContainer outputContainer = outputCommitters.get(alias); + needTaskCommit = needTaskCommit + || outputContainer.getBaseCommitter().needsTaskCommit( + outputContainer.getContext()); + } + return needTaskCommit; + } + + @Override + public void commitTask(TaskAttemptContext taskContext) throws IOException { + for (String alias : outputCommitters.keySet()) { + BaseOutputCommitterContainer outputContainer = outputCommitters.get(alias); + OutputCommitter baseCommitter = outputContainer.getBaseCommitter(); + TaskAttemptContext committerContext = outputContainer.getContext(); + if (baseCommitter.needsTaskCommit(committerContext)) { + LOGGER.info("Calling commitTask for alias: " + alias); + baseCommitter.commitTask(committerContext); + } + } + } + + @Override + public void abortTask(TaskAttemptContext taskContext) throws IOException { + for (String alias : outputCommitters.keySet()) { + LOGGER.info("Calling abortTask for alias: " + alias); + BaseOutputCommitterContainer outputContainer = outputCommitters.get(alias); + outputContainer.getBaseCommitter().abortTask(outputContainer.getContext()); + } + } + + @Override + public void commitJob(JobContext jobContext) throws IOException { + for (String alias : outputCommitters.keySet()) { + LOGGER.info("Calling commitJob for alias: " + alias); + BaseOutputCommitterContainer outputContainer = outputCommitters.get(alias); + outputContainer.getBaseCommitter().commitJob(outputContainer.getContext()); + } + } + + @Override + public void abortJob(JobContext jobContext, State state) throws IOException { + for (String alias : outputCommitters.keySet()) { + LOGGER.info("Calling abortJob for alias: " + alias); + BaseOutputCommitterContainer outputContainer = outputCommitters.get(alias); + outputContainer.getBaseCommitter().abortJob(outputContainer.getContext(), state); + } + } + } + + private static class BaseOutputCommitterContainer { + + private final OutputCommitter outputCommitter; + private final TaskAttemptContext context; + + public BaseOutputCommitterContainer(OutputCommitter outputCommitter, + TaskAttemptContext context) { + this.outputCommitter = outputCommitter; + this.context = context; + } + + public OutputCommitter getBaseCommitter() { + return outputCommitter; + } + + public TaskAttemptContext getContext() { + return context; + } + } + +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/OutputCommitterContainer.java hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/OutputCommitterContainer.java new file mode 100644 index 0000000..1caf5b3 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/OutputCommitterContainer.java @@ -0,0 +1,48 @@ +/** + * 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.hcatalog.mapreduce; + +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.OutputCommitter; + +/** + * This class will contain an implementation of an OutputCommitter. + * See {@link OutputFormatContainer} for more information about containers. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.OutputCommitterContainer} instead + */ +abstract class OutputCommitterContainer extends OutputCommitter { + private final org.apache.hadoop.mapred.OutputCommitter committer; + + /** + * @param context current JobContext + * @param committer OutputCommitter that this instance will contain + */ + public OutputCommitterContainer(JobContext context, org.apache.hadoop.mapred.OutputCommitter committer) { + this.committer = committer; + } + + /** + * @return underlying OutputCommitter + */ + public OutputCommitter getBaseOutputCommitter() { + return committer; + } + +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/OutputFormatContainer.java hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/OutputFormatContainer.java new file mode 100644 index 0000000..77b6cc0 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/OutputFormatContainer.java @@ -0,0 +1,54 @@ +/** + * 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.hcatalog.mapreduce; + +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.io.WritableComparable; +import org.apache.hadoop.mapreduce.OutputFormat; +import org.apache.hcatalog.data.HCatRecord; + +/** + * This container class is used to wrap OutputFormat implementations and augment them with + * behavior necessary to work with HCatalog (ie metastore updates, hcatalog delegation tokens, etc). + * Containers are also used to provide storage specific implementations of some HCatalog features (ie dynamic partitioning). + * Hence users wishing to create storage specific implementations of HCatalog features should implement this class and override + * HCatStorageHandler.getOutputFormatContainer(OutputFormat outputFormat) to return the implementation. + * By default DefaultOutputFormatContainer is used, which only implements the bare minimum features HCatalog features + * such as partitioning isn't supported. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.OutputFormatContainer} instead + */ +abstract class OutputFormatContainer extends OutputFormat, HCatRecord> { + private org.apache.hadoop.mapred.OutputFormat, ? super Writable> of; + + /** + * @param of OutputFormat this instance will contain + */ + public OutputFormatContainer(org.apache.hadoop.mapred.OutputFormat, ? super Writable> of) { + this.of = of; + } + + /** + * @return underlying OutputFormat + */ + public org.apache.hadoop.mapred.OutputFormat getBaseOutputFormat() { + return of; + } + +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/OutputJobInfo.java hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/OutputJobInfo.java new file mode 100644 index 0000000..24a3323 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/OutputJobInfo.java @@ -0,0 +1,273 @@ +/** + * 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.hcatalog.mapreduce; + +import java.io.Serializable; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import org.apache.hadoop.hive.metastore.MetaStoreUtils; +import org.apache.hcatalog.data.schema.HCatSchema; + +/** + * The class used to serialize and store the output related information + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.OutputJobInfo} instead + */ +public class OutputJobInfo implements Serializable { + + /** The db and table names. */ + private final String databaseName; + private final String tableName; + + /** The serialization version. */ + private static final long serialVersionUID = 1L; + + /** The table info provided by user. */ + private HCatTableInfo tableInfo; + + /** The output schema. This is given to us by user. This wont contain any + * partition columns ,even if user has specified them. + * */ + private HCatSchema outputSchema; + + /** The location of the partition being written */ + private String location; + + /** The partition values to publish to, if used for output*/ + private Map partitionValues; + + private List posOfPartCols; + private List posOfDynPartCols; + + private Properties properties; + + private int maxDynamicPartitions; + + /** List of keys for which values were not specified at write setup time, to be infered at write time */ + private List dynamicPartitioningKeys; + + private boolean harRequested; + + /** + * Initializes a new OutputJobInfo instance + * for writing data from a table. + * @param databaseName the db name + * @param tableName the table name + * @param partitionValues The partition values to publish to, can be null or empty Map to + * work with hadoop security, the kerberos principal name of the server - else null + * The principal name should be of the form: + * /_HOST@ like "hcat/_HOST@myrealm.com" + * The special string _HOST will be replaced automatically with the correct host name + * indicate write to a unpartitioned table. For partitioned tables, this map should + * contain keys for all partition columns with corresponding values. + */ + public static OutputJobInfo create(String databaseName, + String tableName, + Map partitionValues) { + return new OutputJobInfo(databaseName, + tableName, + partitionValues); + } + + private OutputJobInfo(String databaseName, + String tableName, + Map partitionValues) { + this.databaseName = (databaseName == null) ? MetaStoreUtils.DEFAULT_DATABASE_NAME : databaseName; + this.tableName = tableName; + this.partitionValues = partitionValues; + this.properties = new Properties(); + } + + /** + * @return the posOfPartCols + */ + protected List getPosOfPartCols() { + return posOfPartCols; + } + + /** + * @return the posOfDynPartCols + */ + protected List getPosOfDynPartCols() { + return posOfDynPartCols; + } + + /** + * @param posOfPartCols the posOfPartCols to set + */ + protected void setPosOfPartCols(List posOfPartCols) { + // sorting the list in the descending order so that deletes happen back-to-front + Collections.sort(posOfPartCols, new Comparator() { + @Override + public int compare(Integer earlier, Integer later) { + return (earlier > later) ? -1 : ((earlier == later) ? 0 : 1); + } + }); + this.posOfPartCols = posOfPartCols; + } + + /** + * @param posOfDynPartCols the posOfDynPartCols to set + */ + protected void setPosOfDynPartCols(List posOfDynPartCols) { + // Important - no sorting here! We retain order, it's used to match with values at runtime + this.posOfDynPartCols = posOfDynPartCols; + } + + /** + * @return the tableInfo + */ + public HCatTableInfo getTableInfo() { + return tableInfo; + } + + /** + * @return the outputSchema + */ + public HCatSchema getOutputSchema() { + return outputSchema; + } + + /** + * @param schema the outputSchema to set + */ + public void setOutputSchema(HCatSchema schema) { + this.outputSchema = schema; + } + + /** + * @return the location + */ + public String getLocation() { + return location; + } + + /** + * @param location location to write to + */ + public void setLocation(String location) { + this.location = location; + } + + /** + * Sets the value of partitionValues + * @param partitionValues the partition values to set + */ + void setPartitionValues(Map partitionValues) { + this.partitionValues = partitionValues; + } + + /** + * Gets the value of partitionValues + * @return the partitionValues + */ + public Map getPartitionValues() { + return partitionValues; + } + + /** + * set the tablInfo instance + * this should be the same instance + * determined by this object's DatabaseName and TableName + * @param tableInfo + */ + void setTableInfo(HCatTableInfo tableInfo) { + this.tableInfo = tableInfo; + } + + /** + * @return database name of table to write to + */ + public String getDatabaseName() { + return databaseName; + } + + /** + * @return name of table to write to + */ + public String getTableName() { + return tableName; + } + + /** + * Set/Get Property information to be passed down to *StorageHandler implementation + * put implementation specific storage handler configurations here + * @return the implementation specific job properties + */ + public Properties getProperties() { + return properties; + } + + /** + * Set maximum number of allowable dynamic partitions + * @param maxDynamicPartitions + */ + public void setMaximumDynamicPartitions(int maxDynamicPartitions) { + this.maxDynamicPartitions = maxDynamicPartitions; + } + + /** + * Returns maximum number of allowable dynamic partitions + * @return maximum number of allowable dynamic partitions + */ + public int getMaxDynamicPartitions() { + return this.maxDynamicPartitions; + } + + /** + * Sets whether or not hadoop archiving has been requested for this job + * @param harRequested + */ + public void setHarRequested(boolean harRequested) { + this.harRequested = harRequested; + } + + /** + * Returns whether or not hadoop archiving has been requested for this job + * @return whether or not hadoop archiving has been requested for this job + */ + public boolean getHarRequested() { + return this.harRequested; + } + + /** + * Returns whether or not Dynamic Partitioning is used + * @return whether or not dynamic partitioning is currently enabled and used + */ + public boolean isDynamicPartitioningUsed() { + return !((dynamicPartitioningKeys == null) || (dynamicPartitioningKeys.isEmpty())); + } + + /** + * Sets the list of dynamic partitioning keys used for outputting without specifying all the keys + * @param dynamicPartitioningKeys + */ + public void setDynamicPartitioningKeys(List dynamicPartitioningKeys) { + this.dynamicPartitioningKeys = dynamicPartitioningKeys; + } + + public List getDynamicPartitioningKeys() { + return this.dynamicPartitioningKeys; + } + +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/PartInfo.java hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/PartInfo.java new file mode 100644 index 0000000..604a36b --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/PartInfo.java @@ -0,0 +1,167 @@ +/** + * 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.hcatalog.mapreduce; + +import java.io.Serializable; +import java.util.Map; +import java.util.Properties; + +import org.apache.hcatalog.data.schema.HCatSchema; + +/** + * The Class used to serialize the partition information read from the metadata server that maps to a partition. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.PartInfo} instead + */ +public class PartInfo implements Serializable { + + /** The serialization version */ + private static final long serialVersionUID = 1L; + + /** The partition schema. */ + private final HCatSchema partitionSchema; + + /** The information about which input storage handler to use */ + private final String storageHandlerClassName; + private final String inputFormatClassName; + private final String outputFormatClassName; + private final String serdeClassName; + + /** HCat-specific properties set at the partition */ + private final Properties hcatProperties; + + /** The data location. */ + private final String location; + + /** The map of partition key names and their values. */ + private Map partitionValues; + + /** Job properties associated with this parition */ + Map jobProperties; + + /** the table info associated with this partition */ + HCatTableInfo tableInfo; + + /** + * Instantiates a new hcat partition info. + * @param partitionSchema the partition schema + * @param storageHandler the storage handler + * @param location the location + * @param hcatProperties hcat-specific properties at the partition + * @param jobProperties the job properties + * @param tableInfo the table information + */ + public PartInfo(HCatSchema partitionSchema, HCatStorageHandler storageHandler, + String location, Properties hcatProperties, + Map jobProperties, HCatTableInfo tableInfo) { + this.partitionSchema = partitionSchema; + this.location = location; + this.hcatProperties = hcatProperties; + this.jobProperties = jobProperties; + this.tableInfo = tableInfo; + + this.storageHandlerClassName = storageHandler.getClass().getName(); + this.inputFormatClassName = storageHandler.getInputFormatClass().getName(); + this.serdeClassName = storageHandler.getSerDeClass().getName(); + this.outputFormatClassName = storageHandler.getOutputFormatClass().getName(); + } + + /** + * Gets the value of partitionSchema. + * @return the partitionSchema + */ + public HCatSchema getPartitionSchema() { + return partitionSchema; + } + + /** + * @return the storage handler class name + */ + public String getStorageHandlerClassName() { + return storageHandlerClassName; + } + + /** + * @return the inputFormatClassName + */ + public String getInputFormatClassName() { + return inputFormatClassName; + } + + /** + * @return the outputFormatClassName + */ + public String getOutputFormatClassName() { + return outputFormatClassName; + } + + /** + * @return the serdeClassName + */ + public String getSerdeClassName() { + return serdeClassName; + } + + /** + * Gets the input storage handler properties. + * @return HCat-specific properties set at the partition + */ + public Properties getInputStorageHandlerProperties() { + return hcatProperties; + } + + /** + * Gets the value of location. + * @return the location + */ + public String getLocation() { + return location; + } + + /** + * Sets the partition values. + * @param partitionValues the new partition values + */ + public void setPartitionValues(Map partitionValues) { + this.partitionValues = partitionValues; + } + + /** + * Gets the partition values. + * @return the partition values + */ + public Map getPartitionValues() { + return partitionValues; + } + + /** + * Gets the job properties. + * @return a map of the job properties + */ + public Map getJobProperties() { + return jobProperties; + } + + /** + * Gets the HCatalog table information. + * @return the table information + */ + public HCatTableInfo getTableInfo() { + return tableInfo; + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/ProgressReporter.java hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/ProgressReporter.java new file mode 100644 index 0000000..a95f997 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/ProgressReporter.java @@ -0,0 +1,95 @@ +/** + * 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.hcatalog.mapreduce; + +import org.apache.hadoop.mapred.Counters; +import org.apache.hadoop.mapred.InputSplit; +import org.apache.hadoop.mapred.Reporter; +import org.apache.hadoop.mapreduce.StatusReporter; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.TaskInputOutputContext; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.ProgressReporter} instead + */ +class ProgressReporter extends StatusReporter implements Reporter { + + private TaskInputOutputContext context = null; + private TaskAttemptContext taskAttemptContext = null; + + public ProgressReporter(TaskAttemptContext context) { + if (context instanceof TaskInputOutputContext) { + this.context = (TaskInputOutputContext) context; + } else { + taskAttemptContext = context; + } + } + + @Override + public void setStatus(String status) { + if (context != null) { + context.setStatus(status); + } + } + + @Override + public Counters.Counter getCounter(Enum name) { + return (context != null) ? (Counters.Counter) context.getCounter(name) : null; + } + + @Override + public Counters.Counter getCounter(String group, String name) { + return (context != null) ? (Counters.Counter) context.getCounter(group, name) : null; + } + + @Override + public void incrCounter(Enum key, long amount) { + if (context != null) { + context.getCounter(key).increment(amount); + } + } + + @Override + public void incrCounter(String group, String counter, long amount) { + if (context != null) { + context.getCounter(group, counter).increment(amount); + } + } + + @Override + public InputSplit getInputSplit() throws UnsupportedOperationException { + return null; + } + + public float getProgress() { + /* Required to build against 0.23 Reporter and StatusReporter. */ + /* TODO: determine the progress. */ + return 0.0f; + } + + @Override + public void progress() { + if (context != null) { + context.progress(); + } else { + taskAttemptContext.progress(); + } + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/RecordWriterContainer.java hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/RecordWriterContainer.java new file mode 100644 index 0000000..69d3abc --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/RecordWriterContainer.java @@ -0,0 +1,54 @@ +/** + * 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.hcatalog.mapreduce; + + +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.io.WritableComparable; +import org.apache.hadoop.mapreduce.RecordWriter; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hcatalog.data.HCatRecord; + +/** + * This class will contain an implementation of an RecordWriter. + * See {@link OutputFormatContainer} for more information about containers. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.RecordWriterContainer} instead + */ +abstract class RecordWriterContainer extends RecordWriter, HCatRecord> { + + private final org.apache.hadoop.mapred.RecordWriter, ? super Writable> baseRecordWriter; + + /** + * @param context current JobContext + * @param baseRecordWriter RecordWriter that this instance will contain + */ + public RecordWriterContainer(TaskAttemptContext context, + org.apache.hadoop.mapred.RecordWriter, ? super Writable> baseRecordWriter) { + this.baseRecordWriter = baseRecordWriter; + } + + /** + * @return underlying RecordWriter + */ + public org.apache.hadoop.mapred.RecordWriter getBaseRecordWriter() { + return baseRecordWriter; + } + +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/Security.java hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/Security.java new file mode 100644 index 0000000..6221a04 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/Security.java @@ -0,0 +1,194 @@ +/** + * 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.hcatalog.mapreduce; + +import java.io.IOException; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.thrift.DelegationTokenSelector; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.security.Credentials; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.security.token.TokenIdentifier; +import org.apache.hadoop.security.token.TokenSelector; +import org.apache.hcatalog.common.HCatConstants; +import org.apache.hcatalog.common.HCatUtil; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.Security} instead + */ +final class Security { + + private static final Logger LOG = LoggerFactory.getLogger(HCatOutputFormat.class); + + // making sure this is not initialized unless needed + private static final class LazyHolder { + public static final Security INSTANCE = new Security(); + } + + public static Security getInstance() { + return LazyHolder.INSTANCE; + } + + boolean isSecurityEnabled() { + try { + Method m = UserGroupInformation.class.getMethod("isSecurityEnabled"); + return (Boolean) m.invoke(null, (Object[]) null); + } catch (NoSuchMethodException e) { + LOG.info("Security is not supported by this version of hadoop.", e); + } catch (InvocationTargetException e) { + String msg = "Failed to call isSecurityEnabled()"; + LOG.info(msg, e); + throw new IllegalStateException(msg, e); + } catch (IllegalAccessException e) { + String msg = "Failed to call isSecurityEnabled()"; + LOG.info(msg, e); + throw new IllegalStateException(msg, e); + } + return false; + } + + // a signature string to associate with a HCatTableInfo - essentially + // a concatenation of dbname, tablename and partition keyvalues. + String getTokenSignature(OutputJobInfo outputJobInfo) { + StringBuilder result = new StringBuilder(""); + String dbName = outputJobInfo.getDatabaseName(); + if (dbName != null) { + result.append(dbName); + } + String tableName = outputJobInfo.getTableName(); + if (tableName != null) { + result.append("." + tableName); + } + Map partValues = outputJobInfo.getPartitionValues(); + if (partValues != null) { + for (Entry entry : partValues.entrySet()) { + result.append("/"); + result.append(entry.getKey()); + result.append("="); + result.append(entry.getValue()); + } + + } + return result.toString(); + } + + void handleSecurity( + Credentials credentials, + OutputJobInfo outputJobInfo, + HiveMetaStoreClient client, + Configuration conf, + boolean harRequested) + throws IOException, MetaException, TException, Exception { + if (UserGroupInformation.isSecurityEnabled()) { + UserGroupInformation ugi = UserGroupInformation.getCurrentUser(); + // check if oozie has set up a hcat deleg. token - if so use it + TokenSelector hiveTokenSelector = new DelegationTokenSelector(); + //Oozie does not change the service field of the token + //hence by default token generation will have a value of "new Text("")" + //HiveClient will look for a use TokenSelector.selectToken() with service + //set to empty "Text" if hive.metastore.token.signature property is set to null + Token hiveToken = hiveTokenSelector.selectToken( + new Text(), ugi.getTokens()); + if (hiveToken == null) { + // we did not get token set up by oozie, let's get them ourselves here. + // we essentially get a token per unique Output HCatTableInfo - this is + // done because through Pig, setOutput() method is called multiple times + // We want to only get the token once per unique output HCatTableInfo - + // we cannot just get one token since in multi-query case (> 1 store in 1 job) + // or the case when a single pig script results in > 1 jobs, the single + // token will get cancelled by the output committer and the subsequent + // stores will fail - by tying the token with the concatenation of + // dbname, tablename and partition keyvalues of the output + // TableInfo, we can have as many tokens as there are stores and the TokenSelector + // will correctly pick the right tokens which the committer will use and + // cancel. + String tokenSignature = getTokenSignature(outputJobInfo); + // get delegation tokens from hcat server and store them into the "job" + // These will be used in to publish partitions to + // hcat normally in OutputCommitter.commitJob() + // when the JobTracker in Hadoop MapReduce starts supporting renewal of + // arbitrary tokens, the renewer should be the principal of the JobTracker + hiveToken = HCatUtil.extractThriftToken(client.getDelegationToken(ugi.getUserName()), tokenSignature); + + if (harRequested) { + TokenSelector jtTokenSelector = + new org.apache.hadoop.mapreduce.security.token.delegation.DelegationTokenSelector(); + Token jtToken = jtTokenSelector.selectToken(org.apache.hadoop.security.SecurityUtil.buildTokenService( + ShimLoader.getHadoopShims().getHCatShim().getResourceManagerAddress(conf)), ugi.getTokens()); + if (jtToken == null) { + //we don't need to cancel this token as the TokenRenewer for JT tokens + //takes care of cancelling them + credentials.addToken( + new Text("hcat jt token"), + HCatUtil.getJobTrackerDelegationToken(conf, ugi.getUserName()) + ); + } + } + + credentials.addToken(new Text(ugi.getUserName() + "_" + tokenSignature), hiveToken); + // this will be used by the outputcommitter to pass on to the metastore client + // which in turn will pass on to the TokenSelector so that it can select + // the right token. + conf.set(HCatConstants.HCAT_KEY_TOKEN_SIGNATURE, tokenSignature); + } + } + } + + void handleSecurity( + Job job, + OutputJobInfo outputJobInfo, + HiveMetaStoreClient client, + Configuration conf, + boolean harRequested) + throws IOException, MetaException, TException, Exception { + handleSecurity(job.getCredentials(), outputJobInfo, client, conf, harRequested); + } + + // we should cancel hcat token if it was acquired by hcat + // and not if it was supplied (ie Oozie). In the latter + // case the HCAT_KEY_TOKEN_SIGNATURE property in the conf will not be set + void cancelToken(HiveMetaStoreClient client, JobContext context) throws IOException, MetaException { + String tokenStrForm = client.getTokenStrForm(); + if (tokenStrForm != null && context.getConfiguration().get(HCatConstants.HCAT_KEY_TOKEN_SIGNATURE) != null) { + try { + client.cancelDelegationToken(tokenStrForm); + } catch (TException e) { + String msg = "Failed to cancel delegation token"; + LOG.error(msg, e); + throw new IOException(msg, e); + } + } + } + +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/StorerInfo.java hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/StorerInfo.java new file mode 100644 index 0000000..446f181 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/mapreduce/StorerInfo.java @@ -0,0 +1,112 @@ +/** + * 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.hcatalog.mapreduce; + +import java.io.Serializable; +import java.util.Properties; + +/** + * Information about the storer to use for writing the data. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.StorerInfo} instead + */ +public class StorerInfo implements Serializable { + + /** The serialization version */ + private static final long serialVersionUID = 1L; + + /** The properties for the storage handler */ + private Properties properties; + + private String ofClass; + + private String ifClass; + + private String serdeClass; + + private String storageHandlerClass; + + /** + * Initialize the storer information. + * @param ifClass the input format class + * @param ofClass the output format class + * @param serdeClass the SerDe class + * @param storageHandlerClass the storage handler class + * @param properties the properties for the storage handler + */ + public StorerInfo(String ifClass, String ofClass, String serdeClass, String storageHandlerClass, Properties properties) { + super(); + this.ifClass = ifClass; + this.ofClass = ofClass; + this.serdeClass = serdeClass; + this.storageHandlerClass = storageHandlerClass; + this.properties = properties; + } + + /** + * @return the input format class + */ + public String getIfClass() { + return ifClass; + } + + /** + * @param ifClass the input format class + */ + public void setIfClass(String ifClass) { + this.ifClass = ifClass; + } + + /** + * @return the output format class + */ + public String getOfClass() { + return ofClass; + } + + /** + * @return the serdeClass + */ + public String getSerdeClass() { + return serdeClass; + } + + /** + * @return the storageHandlerClass + */ + public String getStorageHandlerClass() { + return storageHandlerClass; + } + + /** + * @return the storer properties + */ + public Properties getProperties() { + return properties; + } + + /** + * @param properties the storer properties to set + */ + public void setProperties(Properties properties) { + this.properties = properties; + } + + +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/oozie/JavaAction.java hcatalog/core/src/main/java/org/apache/hcatalog/oozie/JavaAction.java new file mode 100644 index 0000000..dcc7fed --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/oozie/JavaAction.java @@ -0,0 +1,44 @@ +/** + * 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.hcatalog.oozie; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.cli.CliDriver; +import org.apache.hadoop.hive.cli.CliSessionState; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hcatalog.cli.SemanticAnalysis.HCatSemanticAnalyzer; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.oozie.JavaAction} instead + */ +public class JavaAction { + + public static void main(String[] args) throws Exception { + + HiveConf conf = new HiveConf(); + conf.addResource(new Path("file:///", System.getProperty("oozie.action.conf.xml"))); + conf.setVar(ConfVars.SEMANTIC_ANALYZER_HOOK, HCatSemanticAnalyzer.class.getName()); + conf.setBoolVar(ConfVars.METASTORE_USE_THRIFT_SASL, true); + SessionState.start(new CliSessionState(conf)); + new CliDriver().processLine(args[0]); + } + +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/rcfile/RCFileMapReduceInputFormat.java hcatalog/core/src/main/java/org/apache/hcatalog/rcfile/RCFileMapReduceInputFormat.java new file mode 100644 index 0000000..a61d07a --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/rcfile/RCFileMapReduceInputFormat.java @@ -0,0 +1,53 @@ +/** + * 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.hcatalog.rcfile; + +import java.io.IOException; +import java.util.List; + +import org.apache.hadoop.hive.serde2.columnar.BytesRefArrayWritable; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.mapreduce.InputSplit; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.RecordReader; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.rcfile.RCFileMapReduceInputFormat} instead + */ +public class RCFileMapReduceInputFormat + extends FileInputFormat { + + @Override + public RecordReader createRecordReader(InputSplit split, + TaskAttemptContext context) throws IOException, InterruptedException { + + context.setStatus(split.toString()); + return new RCFileMapReduceRecordReader(); + } + + @Override + public List getSplits(JobContext job) throws IOException { + + job.getConfiguration().setLong("mapred.min.split.size", SequenceFile.SYNC_INTERVAL); + return super.getSplits(job); + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/rcfile/RCFileMapReduceOutputFormat.java hcatalog/core/src/main/java/org/apache/hcatalog/rcfile/RCFileMapReduceOutputFormat.java new file mode 100644 index 0000000..a0775dc --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/rcfile/RCFileMapReduceOutputFormat.java @@ -0,0 +1,106 @@ +/** + * 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.hcatalog.rcfile; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.ql.io.RCFile; +import org.apache.hadoop.hive.serde2.columnar.BytesRefArrayWritable; +import org.apache.hadoop.io.WritableComparable; +import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.hadoop.io.compress.DefaultCodec; +import org.apache.hadoop.mapreduce.RecordWriter; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter; +import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; +import org.apache.hadoop.util.ReflectionUtils; + +/** + * The RC file input format using new Hadoop mapreduce APIs. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.rcfile.RCFileMapReduceOutputFormat} instead + */ +public class RCFileMapReduceOutputFormat extends + FileOutputFormat, BytesRefArrayWritable> { + + /** + * Set number of columns into the given configuration. + * @param conf + * configuration instance which need to set the column number + * @param columnNum + * column number for RCFile's Writer + * + */ + public static void setColumnNumber(Configuration conf, int columnNum) { + assert columnNum > 0; + conf.setInt(RCFile.COLUMN_NUMBER_CONF_STR, columnNum); + } + + /* (non-Javadoc) + * @see org.apache.hadoop.mapreduce.lib.output.FileOutputFormat#getRecordWriter(org.apache.hadoop.mapreduce.TaskAttemptContext) + */ + @Override + public org.apache.hadoop.mapreduce.RecordWriter, BytesRefArrayWritable> getRecordWriter( + TaskAttemptContext task) throws IOException, InterruptedException { + + //FileOutputFormat.getWorkOutputPath takes TaskInputOutputContext instead of + //TaskAttemptContext, so can't use that here + FileOutputCommitter committer = (FileOutputCommitter) getOutputCommitter(task); + Path outputPath = committer.getWorkPath(); + + FileSystem fs = outputPath.getFileSystem(task.getConfiguration()); + + if (!fs.exists(outputPath)) { + fs.mkdirs(outputPath); + } + + Path file = getDefaultWorkFile(task, ""); + + CompressionCodec codec = null; + if (getCompressOutput(task)) { + Class codecClass = getOutputCompressorClass(task, DefaultCodec.class); + codec = (CompressionCodec) ReflectionUtils.newInstance(codecClass, task.getConfiguration()); + } + + final RCFile.Writer out = new RCFile.Writer(fs, task.getConfiguration(), file, task, codec); + + return new RecordWriter, BytesRefArrayWritable>() { + + /* (non-Javadoc) + * @see org.apache.hadoop.mapreduce.RecordWriter#write(java.lang.Object, java.lang.Object) + */ + @Override + public void write(WritableComparable key, BytesRefArrayWritable value) + throws IOException { + out.append(value); + } + + /* (non-Javadoc) + * @see org.apache.hadoop.mapreduce.RecordWriter#close(org.apache.hadoop.mapreduce.TaskAttemptContext) + */ + @Override + public void close(TaskAttemptContext task) throws IOException, InterruptedException { + out.close(); + } + }; + } + +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/rcfile/RCFileMapReduceRecordReader.java hcatalog/core/src/main/java/org/apache/hcatalog/rcfile/RCFileMapReduceRecordReader.java new file mode 100644 index 0000000..1f82501 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/rcfile/RCFileMapReduceRecordReader.java @@ -0,0 +1,123 @@ +/** + * 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.hcatalog.rcfile; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.ql.io.RCFile; +import org.apache.hadoop.hive.ql.io.RCFile.Reader; +import org.apache.hadoop.hive.serde2.columnar.BytesRefArrayWritable; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.mapreduce.InputSplit; +import org.apache.hadoop.mapreduce.RecordReader; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.lib.input.FileSplit; +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.rcfile.RCFileMapReduceRecordReader} instead + */ +public class RCFileMapReduceRecordReader + extends RecordReader { + + private Reader in; + private long start; + private long end; + private boolean more = true; + + // key and value objects are created once in initialize() and then reused + // for every getCurrentKey() and getCurrentValue() call. This is important + // since RCFile makes an assumption of this fact. + + private LongWritable key; + private BytesRefArrayWritable value; + + @Override + public void close() throws IOException { + in.close(); + } + + @Override + public LongWritable getCurrentKey() throws IOException, InterruptedException { + return key; + } + + @Override + public BytesRefArrayWritable getCurrentValue() throws IOException, InterruptedException { + return value; + } + + @Override + public float getProgress() throws IOException, InterruptedException { + if (end == start) { + return 0.0f; + } else { + return Math.min(1.0f, (in.getPosition() - start) / (float) (end - start)); + } + } + + @Override + public boolean nextKeyValue() throws IOException, InterruptedException { + + more = next(key); + if (more) { + in.getCurrentRow(value); + } + + return more; + } + + private boolean next(LongWritable key) throws IOException { + if (!more) { + return false; + } + + more = in.next(key); + if (!more) { + return false; + } + + if (in.lastSeenSyncPos() >= end) { + more = false; + return more; + } + return more; + } + + @Override + public void initialize(InputSplit split, TaskAttemptContext context) throws IOException, + InterruptedException { + + FileSplit fSplit = (FileSplit) split; + Path path = fSplit.getPath(); + Configuration conf = context.getConfiguration(); + this.in = new RCFile.Reader(path.getFileSystem(conf), path, conf); + this.end = fSplit.getStart() + fSplit.getLength(); + + if (fSplit.getStart() > in.getPosition()) { + in.sync(fSplit.getStart()); + } + + this.start = in.getPosition(); + more = start < end; + + key = new LongWritable(); + value = new BytesRefArrayWritable(); + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/security/HdfsAuthorizationProvider.java hcatalog/core/src/main/java/org/apache/hcatalog/security/HdfsAuthorizationProvider.java new file mode 100644 index 0000000..080e154 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/security/HdfsAuthorizationProvider.java @@ -0,0 +1,338 @@ +/** + * 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.hcatalog.security; + +import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.EnumSet; +import java.util.List; + +import javax.security.auth.login.LoginException; + +import org.apache.commons.lang.ArrayUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsAction; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.Warehouse; +import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.ql.metadata.AuthorizationException; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.metadata.Partition; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider; +import org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProviderBase; +import org.apache.hadoop.hive.ql.security.authorization.Privilege; +import org.apache.hadoop.hive.shims.HadoopShims; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hadoop.security.AccessControlException; +import org.apache.hadoop.security.UserGroupInformation; + +/** + * An AuthorizationProvider, which checks against the data access level permissions on HDFS. + * It makes sense to eventually move this class to Hive, so that all hive users can + * use this authorization model. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.security.HdfsAuthorizationProvider} instead + */ +public class HdfsAuthorizationProvider extends HiveAuthorizationProviderBase { + + protected Warehouse wh; + + //Config variables : create an enum to store them if we have more + private static final String PROXY_USER_NAME = "proxy.user.name"; + + public HdfsAuthorizationProvider() { + super(); + } + + public HdfsAuthorizationProvider(Configuration conf) { + super(); + setConf(conf); + } + + @Override + public void init(Configuration conf) throws HiveException { + hive_db = new HiveProxy(Hive.get(new HiveConf(conf, HiveAuthorizationProvider.class))); + } + + @Override + public void setConf(Configuration conf) { + super.setConf(conf); + try { + this.wh = new Warehouse(conf); + } catch (MetaException ex) { + throw new RuntimeException(ex); + } + } + + protected FsAction getFsAction(Privilege priv, Path path) { + + switch (priv.getPriv()) { + case ALL: + throw new AuthorizationException("no matching Action for Privilege.All"); + case ALTER_DATA: + return FsAction.WRITE; + case ALTER_METADATA: + return FsAction.WRITE; + case CREATE: + return FsAction.WRITE; + case DROP: + return FsAction.WRITE; + case INDEX: + return FsAction.WRITE; + case LOCK: + return FsAction.WRITE; + case SELECT: + return FsAction.READ; + case SHOW_DATABASE: + return FsAction.READ; + case UNKNOWN: + default: + throw new AuthorizationException("Unknown privilege"); + } + } + + protected EnumSet getFsActions(Privilege[] privs, Path path) { + EnumSet actions = EnumSet.noneOf(FsAction.class); + + if (privs == null) { + return actions; + } + + for (Privilege priv : privs) { + actions.add(getFsAction(priv, path)); + } + + return actions; + } + + private static final String DATABASE_WAREHOUSE_SUFFIX = ".db"; + + private Path getDefaultDatabasePath(String dbName) throws MetaException { + if (dbName.equalsIgnoreCase(DEFAULT_DATABASE_NAME)) { + return wh.getWhRoot(); + } + return new Path(wh.getWhRoot(), dbName.toLowerCase() + DATABASE_WAREHOUSE_SUFFIX); + } + + protected Path getDbLocation(Database db) throws HiveException { + try { + String location = db.getLocationUri(); + if (location == null) { + return getDefaultDatabasePath(db.getName()); + } else { + return wh.getDnsPath(wh.getDatabasePath(db)); + } + } catch (MetaException ex) { + throw new HiveException(ex.getMessage()); + } + } + + @Override + public void authorize(Privilege[] readRequiredPriv, Privilege[] writeRequiredPriv) + throws HiveException, AuthorizationException { + //Authorize for global level permissions at the warehouse dir + Path root; + try { + root = wh.getWhRoot(); + authorize(root, readRequiredPriv, writeRequiredPriv); + } catch (MetaException ex) { + throw new HiveException(ex); + } + } + + @Override + public void authorize(Database db, Privilege[] readRequiredPriv, Privilege[] writeRequiredPriv) + throws HiveException, AuthorizationException { + if (db == null) { + return; + } + + Path path = getDbLocation(db); + + authorize(path, readRequiredPriv, writeRequiredPriv); + } + + @Override + public void authorize(Table table, Privilege[] readRequiredPriv, Privilege[] writeRequiredPriv) + throws HiveException, AuthorizationException { + if (table == null) { + return; + } + + //unlike Hive's model, this can be called at CREATE TABLE as well, since we should authorize + //against the table's declared location + Path path = null; + try { + if (table.getTTable().getSd().getLocation() == null + || table.getTTable().getSd().getLocation().isEmpty()) { + path = wh.getTablePath(hive_db.getDatabase(table.getDbName()), table.getTableName()); + } else { + path = table.getPath(); + } + } catch (MetaException ex) { + throw new HiveException(ex); + } + + authorize(path, readRequiredPriv, writeRequiredPriv); + } + + //TODO: HiveAuthorizationProvider should expose this interface instead of #authorize(Partition, Privilege[], Privilege[]) + public void authorize(Table table, Partition part, Privilege[] readRequiredPriv, Privilege[] writeRequiredPriv) + throws HiveException, AuthorizationException { + + if (part == null || part.getLocation() == null) { + authorize(table, readRequiredPriv, writeRequiredPriv); + } else { + authorize(part.getPartitionPath(), readRequiredPriv, writeRequiredPriv); + } + } + + @Override + public void authorize(Partition part, Privilege[] readRequiredPriv, Privilege[] writeRequiredPriv) + throws HiveException, AuthorizationException { + if (part == null) { + return; + } + authorize(part.getTable(), part, readRequiredPriv, writeRequiredPriv); + } + + @Override + public void authorize(Table table, Partition part, List columns, + Privilege[] readRequiredPriv, Privilege[] writeRequiredPriv) throws HiveException, + AuthorizationException { + //columns cannot live in different files, just check for partition level permissions + authorize(table, part, readRequiredPriv, writeRequiredPriv); + } + + /** + * Authorization privileges against a path. + * @param path a filesystem path + * @param readRequiredPriv a list of privileges needed for inputs. + * @param writeRequiredPriv a list of privileges needed for outputs. + */ + public void authorize(Path path, Privilege[] readRequiredPriv, Privilege[] writeRequiredPriv) + throws HiveException, AuthorizationException { + try { + EnumSet actions = getFsActions(readRequiredPriv, path); + actions.addAll(getFsActions(writeRequiredPriv, path)); + if (actions.isEmpty()) { + return; + } + + checkPermissions(getConf(), path, actions); + + } catch (AccessControlException ex) { + throw new AuthorizationException(ex); + } catch (LoginException ex) { + throw new AuthorizationException(ex); + } catch (IOException ex) { + throw new HiveException(ex); + } + } + + /** + * Checks the permissions for the given path and current user on Hadoop FS. If the given path + * does not exists, it checks for it's parent folder. + */ + protected static void checkPermissions(final Configuration conf, final Path path, + final EnumSet actions) throws IOException, LoginException { + + if (path == null) { + throw new IllegalArgumentException("path is null"); + } + + HadoopShims shims = ShimLoader.getHadoopShims(); + final UserGroupInformation ugi; + if (conf.get(PROXY_USER_NAME) != null) { + ugi = UserGroupInformation.createRemoteUser(conf.get(PROXY_USER_NAME)); + } else { + ugi = shims.getUGIForConf(conf); + } + final String user = shims.getShortUserName(ugi); + + final FileSystem fs = path.getFileSystem(conf); + + if (fs.exists(path)) { + checkPermissions(fs, path, actions, user, ugi.getGroupNames()); + } else if (path.getParent() != null) { + // find the ancestor which exists to check it's permissions + Path par = path.getParent(); + while (par != null) { + if (fs.exists(par)) { + break; + } + par = par.getParent(); + } + + checkPermissions(fs, par, actions, user, ugi.getGroupNames()); + } + } + + /** + * Checks the permissions for the given path and current user on Hadoop FS. If the given path + * does not exists, it returns. + */ + @SuppressWarnings("deprecation") + protected static void checkPermissions(final FileSystem fs, final Path path, + final EnumSet actions, String user, String[] groups) throws IOException, + AccessControlException { + + final FileStatus stat; + + try { + stat = fs.getFileStatus(path); + } catch (FileNotFoundException fnfe) { + // File named by path doesn't exist; nothing to validate. + return; + } catch (org.apache.hadoop.fs.permission.AccessControlException ace) { + // Older hadoop version will throw this @deprecated Exception. + throw new AccessControlException(ace.getMessage()); + } + + final FsPermission dirPerms = stat.getPermission(); + final String grp = stat.getGroup(); + + for (FsAction action : actions) { + if (user.equals(stat.getOwner())) { + if (dirPerms.getUserAction().implies(action)) { + continue; + } + } + if (ArrayUtils.contains(groups, grp)) { + if (dirPerms.getGroupAction().implies(action)) { + continue; + } + } + if (dirPerms.getOtherAction().implies(action)) { + continue; + } + throw new AccessControlException("action " + action + " not permitted on path " + + path + " for user " + user); + } + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/security/StorageDelegationAuthorizationProvider.java hcatalog/core/src/main/java/org/apache/hcatalog/security/StorageDelegationAuthorizationProvider.java new file mode 100644 index 0000000..ee75767 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/security/StorageDelegationAuthorizationProvider.java @@ -0,0 +1,143 @@ +/** + * 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.hcatalog.security; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.hadoop.hive.ql.metadata.AuthorizationException; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.metadata.HiveStorageHandler; +import org.apache.hadoop.hive.ql.metadata.Partition; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider; +import org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider; +import org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProviderBase; +import org.apache.hadoop.hive.ql.security.authorization.Privilege; +import org.apache.hadoop.util.ReflectionUtils; +import org.apache.hcatalog.mapreduce.HCatStorageHandler; + +/** + * A HiveAuthorizationProvider which delegates the authorization requests to + * the underlying AuthorizationProviders obtained from the StorageHandler. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.security.StorageDelegationAuthorizationProvider} instead + */ +public class StorageDelegationAuthorizationProvider extends HiveAuthorizationProviderBase { + + protected HiveAuthorizationProvider hdfsAuthorizer = new HdfsAuthorizationProvider(); + + protected static Map authProviders = new HashMap(); + + @Override + public void setConf(Configuration conf) { + super.setConf(conf); + hdfsAuthorizer.setConf(conf); + } + + @Override + public void init(Configuration conf) throws HiveException { + hive_db = new HiveProxy(Hive.get(new HiveConf(conf, HiveAuthorizationProvider.class))); + } + + @Override + public void setAuthenticator(HiveAuthenticationProvider authenticator) { + super.setAuthenticator(authenticator); + hdfsAuthorizer.setAuthenticator(authenticator); + } + + static { + registerAuthProvider("org.apache.hadoop.hive.hbase.HBaseStorageHandler", + "org.apache.hcatalog.hbase.HBaseAuthorizationProvider"); + registerAuthProvider("org.apache.hcatalog.hbase.HBaseHCatStorageHandler", + "org.apache.hcatalog.hbase.HBaseAuthorizationProvider"); + } + + //workaround until Hive adds StorageHandler.getAuthorizationProvider(). Remove these parts afterwards + public static void registerAuthProvider(String storageHandlerClass, + String authProviderClass) { + authProviders.put(storageHandlerClass, authProviderClass); + } + + /** Returns the StorageHandler of the Table obtained from the HCatStorageHandler */ + protected HiveAuthorizationProvider getDelegate(Table table) throws HiveException { + HiveStorageHandler handler = table.getStorageHandler(); + + if (handler != null) { + if (handler instanceof HCatStorageHandler) { + return ((HCatStorageHandler) handler).getAuthorizationProvider(); + } else { + String authProviderClass = authProviders.get(handler.getClass().getCanonicalName()); + + if (authProviderClass != null) { + try { + ReflectionUtils.newInstance(getConf().getClassByName(authProviderClass), getConf()); + } catch (ClassNotFoundException ex) { + throw new HiveException("Cannot instantiate delegation AuthotizationProvider"); + } + } + + //else we do not have anything to delegate to + throw new HiveException(String.format("Storage Handler for table:%s is not an instance " + + "of HCatStorageHandler", table.getTableName())); + } + } else { + //return an authorizer for HDFS + return hdfsAuthorizer; + } + } + + @Override + public void authorize(Privilege[] readRequiredPriv, Privilege[] writeRequiredPriv) + throws HiveException, AuthorizationException { + //global authorizations against warehouse hdfs directory + hdfsAuthorizer.authorize(readRequiredPriv, writeRequiredPriv); + } + + @Override + public void authorize(Database db, Privilege[] readRequiredPriv, Privilege[] writeRequiredPriv) + throws HiveException, AuthorizationException { + //db's are tied to a hdfs location + hdfsAuthorizer.authorize(db, readRequiredPriv, writeRequiredPriv); + } + + @Override + public void authorize(Table table, Privilege[] readRequiredPriv, Privilege[] writeRequiredPriv) + throws HiveException, AuthorizationException { + getDelegate(table).authorize(table, readRequiredPriv, writeRequiredPriv); + } + + @Override + public void authorize(Partition part, Privilege[] readRequiredPriv, + Privilege[] writeRequiredPriv) throws HiveException, AuthorizationException { + getDelegate(part.getTable()).authorize(part, readRequiredPriv, writeRequiredPriv); + } + + @Override + public void authorize(Table table, Partition part, List columns, + Privilege[] readRequiredPriv, Privilege[] writeRequiredPriv) throws HiveException, + AuthorizationException { + getDelegate(table).authorize(table, part, columns, readRequiredPriv, writeRequiredPriv); + } +} diff --git hcatalog/core/src/main/java/org/apache/hcatalog/storagehandler/DummyHCatAuthProvider.java hcatalog/core/src/main/java/org/apache/hcatalog/storagehandler/DummyHCatAuthProvider.java new file mode 100644 index 0000000..b778292 --- /dev/null +++ hcatalog/core/src/main/java/org/apache/hcatalog/storagehandler/DummyHCatAuthProvider.java @@ -0,0 +1,145 @@ +/** + * 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.hcatalog.storagehandler; + +import java.util.List; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.hadoop.hive.ql.metadata.AuthorizationException; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.metadata.Partition; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider; +import org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider; +import org.apache.hadoop.hive.ql.security.authorization.Privilege; + +/** + * This class is a dummy implementation of HiveAuthorizationProvider to provide + * dummy authorization functionality for other classes to extend and override. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.storagehandler.DummyHCatAuthProvider} instead + */ +class DummyHCatAuthProvider implements HiveAuthorizationProvider { + + @Override + public Configuration getConf() { + return null; + } + + @Override + public void setConf(Configuration conf) { + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider + * #init(org.apache.hadoop.conf.Configuration) + */ + @Override + public void init(Configuration conf) throws HiveException { + } + + @Override + public HiveAuthenticationProvider getAuthenticator() { + return null; + } + + @Override + public void setAuthenticator(HiveAuthenticationProvider authenticator) { + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider + * #authorize(org.apache.hadoop.hive.ql.security.authorization.Privilege[], + * org.apache.hadoop.hive.ql.security.authorization.Privilege[]) + */ + @Override + public void authorize(Privilege[] readRequiredPriv, + Privilege[] writeRequiredPriv) throws HiveException, + AuthorizationException { + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider + * #authorize(org.apache.hadoop.hive.metastore.api.Database, + * org.apache.hadoop.hive.ql.security.authorization.Privilege[], + * org.apache.hadoop.hive.ql.security.authorization.Privilege[]) + */ + @Override + public void authorize(Database db, Privilege[] readRequiredPriv, + Privilege[] writeRequiredPriv) throws HiveException, + AuthorizationException { + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider + * #authorize(org.apache.hadoop.hive.ql.metadata.Table, + * org.apache.hadoop.hive.ql.security.authorization.Privilege[], + * org.apache.hadoop.hive.ql.security.authorization.Privilege[]) + */ + @Override + public void authorize(Table table, Privilege[] readRequiredPriv, + Privilege[] writeRequiredPriv) throws HiveException, + AuthorizationException { + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider + * #authorize(org.apache.hadoop.hive.ql.metadata.Partition, + * org.apache.hadoop.hive.ql.security.authorization.Privilege[], + * org.apache.hadoop.hive.ql.security.authorization.Privilege[]) + */ + @Override + public void authorize(Partition part, Privilege[] readRequiredPriv, + Privilege[] writeRequiredPriv) throws HiveException, + AuthorizationException { + } + + /* + * (non-Javadoc) + * + * @see + * org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider + * #authorize(org.apache.hadoop.hive.ql.metadata.Table, + * org.apache.hadoop.hive.ql.metadata.Partition, java.util.List, + * org.apache.hadoop.hive.ql.security.authorization.Privilege[], + * org.apache.hadoop.hive.ql.security.authorization.Privilege[]) + */ + @Override + public void authorize(Table table, Partition part, List columns, + Privilege[] readRequiredPriv, Privilege[] writeRequiredPriv) + throws HiveException, AuthorizationException { + } + +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/ExitException.java hcatalog/core/src/test/java/org/apache/hcatalog/ExitException.java new file mode 100644 index 0000000..0c631d6 --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/ExitException.java @@ -0,0 +1,41 @@ +/** + * 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.hcatalog; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.ExitException} instead + */ +public class ExitException extends SecurityException { + private static final long serialVersionUID = -1982617086752946683L; + private final int status; + + /** + * @return the status + */ + public int getStatus() { + return status; + } + + public ExitException(int status) { + + super("Raising exception, instead of System.exit(). Return code was: " + status); + this.status = status; + } +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/HcatTestUtils.java hcatalog/core/src/test/java/org/apache/hcatalog/HcatTestUtils.java new file mode 100644 index 0000000..67e1050 --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/HcatTestUtils.java @@ -0,0 +1,101 @@ +/** + * 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.hcatalog; + +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hive.metastore.Warehouse; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Utility methods for tests + * @deprecated Use/modify {@link org.apache.hive.hcatalog.HcatTestUtils} instead + */ +public class HcatTestUtils { + private static final Logger LOG = LoggerFactory.getLogger(HcatTestUtils.class); + + public static FsPermission perm007 = FsPermission.createImmutable((short) 0007); // -------rwx + public static FsPermission perm070 = FsPermission.createImmutable((short) 0070); // ----rwx--- + public static FsPermission perm700 = FsPermission.createImmutable((short) 0700); // -rwx------ + public static FsPermission perm755 = FsPermission.createImmutable((short) 0755); // -rwxr-xr-x + public static FsPermission perm777 = FsPermission.createImmutable((short) 0777); // -rwxrwxrwx + public static FsPermission perm300 = FsPermission.createImmutable((short) 0300); // --wx------ + public static FsPermission perm500 = FsPermission.createImmutable((short) 0500); // -r-x------ + public static FsPermission perm555 = FsPermission.createImmutable((short) 0555); // -r-xr-xr-x + + /** + * Returns the database path. + */ + public static Path getDbPath(Hive hive, Warehouse wh, String dbName) throws MetaException, HiveException { + return wh.getDatabasePath(hive.getDatabase(dbName)); + } + + /** + * Removes all databases and tables from the metastore + */ + public static void cleanupHMS(Hive hive, Warehouse wh, FsPermission defaultPerm) + throws HiveException, MetaException, NoSuchObjectException { + for (String dbName : hive.getAllDatabases()) { + if (dbName.equals("default")) { + continue; + } + try { + Path path = getDbPath(hive, wh, dbName); + FileSystem whFs = path.getFileSystem(hive.getConf()); + whFs.setPermission(path, defaultPerm); + } catch (IOException ex) { + //ignore + } + hive.dropDatabase(dbName, true, true, true); + } + + //clean tables in default db + for (String tablename : hive.getAllTables("default")) { + hive.dropTable("default", tablename, true, true); + } + } + + public static void createTestDataFile(String filename, String[] lines) throws IOException { + FileWriter writer = null; + try { + File file = new File(filename); + file.deleteOnExit(); + writer = new FileWriter(file); + for (String line : lines) { + writer.write(line + "\n"); + } + } finally { + if (writer != null) { + writer.close(); + } + } + + } +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/MiniCluster.java hcatalog/core/src/test/java/org/apache/hcatalog/MiniCluster.java new file mode 100644 index 0000000..518d3cc --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/MiniCluster.java @@ -0,0 +1,202 @@ +/** + * 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.hcatalog; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.OutputStreamWriter; +import java.io.PrintWriter; +import java.util.Iterator; +import java.util.Map; +import java.util.Properties; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.MiniMRCluster; + +/** + * This class builds a single instance of itself with the Singleton + * design pattern. While building the single instance, it sets up a + * mini cluster that actually consists of a mini DFS cluster and a + * mini MapReduce cluster on the local machine and also sets up the + * environment for Pig to run on top of the mini cluster. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.MiniCluster} instead + */ +public class MiniCluster { + private MiniDFSCluster m_dfs = null; + private MiniMRCluster m_mr = null; + private FileSystem m_fileSys = null; + private JobConf m_conf = null; + + private final static MiniCluster INSTANCE = new MiniCluster(); + private static boolean isSetup = true; + + private MiniCluster() { + setupMiniDfsAndMrClusters(); + } + + private void setupMiniDfsAndMrClusters() { + try { + final int dataNodes = 1; // There will be 4 data nodes + final int taskTrackers = 1; // There will be 4 task tracker nodes + Configuration config = new Configuration(); + + // Builds and starts the mini dfs and mapreduce clusters + System.setProperty("hadoop.log.dir", "."); + m_dfs = new MiniDFSCluster(config, dataNodes, true, null); + + m_fileSys = m_dfs.getFileSystem(); + m_mr = new MiniMRCluster(taskTrackers, m_fileSys.getUri().toString(), 1); + + // Create the configuration hadoop-site.xml file + File conf_dir = new File(System.getProperty("user.home"), "pigtest/conf/"); + conf_dir.mkdirs(); + File conf_file = new File(conf_dir, "hadoop-site.xml"); + + // Write the necessary config info to hadoop-site.xml + m_conf = m_mr.createJobConf(); + m_conf.setInt("mapred.submit.replication", 1); + m_conf.set("dfs.datanode.address", "0.0.0.0:0"); + m_conf.set("dfs.datanode.http.address", "0.0.0.0:0"); + m_conf.writeXml(new FileOutputStream(conf_file)); + + // Set the system properties needed by Pig + System.setProperty("cluster", m_conf.get("mapred.job.tracker")); + System.setProperty("namenode", m_conf.get("fs.default.name")); + System.setProperty("junit.hadoop.conf", conf_dir.getPath()); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + /** + * Returns the single instance of class MiniClusterBuilder that + * represents the resouces for a mini dfs cluster and a mini + * mapreduce cluster. + */ + public static MiniCluster buildCluster() { + if (!isSetup) { + INSTANCE.setupMiniDfsAndMrClusters(); + isSetup = true; + } + return INSTANCE; + } + + public void shutDown() { + INSTANCE.shutdownMiniDfsAndMrClusters(); + } + + @Override + protected void finalize() { + shutdownMiniDfsAndMrClusters(); + } + + private void shutdownMiniDfsAndMrClusters() { + isSetup = false; + try { + if (m_fileSys != null) { + m_fileSys.close(); + } + } catch (IOException e) { + e.printStackTrace(); + } + if (m_dfs != null) { + m_dfs.shutdown(); + } + if (m_mr != null) { + m_mr.shutdown(); + } + m_fileSys = null; + m_dfs = null; + m_mr = null; + } + + public Properties getProperties() { + errorIfNotSetup(); + Properties properties = new Properties(); + assert m_conf != null; + Iterator> iter = m_conf.iterator(); + while (iter.hasNext()) { + Map.Entry entry = iter.next(); + properties.put(entry.getKey(), entry.getValue()); + } + return properties; + } + + public void setProperty(String name, String value) { + errorIfNotSetup(); + m_conf.set(name, value); + } + + public FileSystem getFileSystem() { + errorIfNotSetup(); + return m_fileSys; + } + + /** + * Throw RunTimeException if isSetup is false + */ + private void errorIfNotSetup() { + if (isSetup) { + return; + } + String msg = "function called on MiniCluster that has been shutdown"; + throw new RuntimeException(msg); + } + + static public void createInputFile(MiniCluster miniCluster, String fileName, + String[] inputData) + throws IOException { + FileSystem fs = miniCluster.getFileSystem(); + createInputFile(fs, fileName, inputData); + } + + static public void createInputFile(FileSystem fs, String fileName, + String[] inputData) throws IOException { + Path path = new Path(fileName); + if (fs.exists(path)) { + throw new IOException("File " + fileName + " already exists on the minicluster"); + } + FSDataOutputStream stream = fs.create(path); + PrintWriter pw = new PrintWriter(new OutputStreamWriter(stream, "UTF-8")); + for (int i = 0; i < inputData.length; i++) { + pw.println(inputData[i]); + } + pw.close(); + + } + + /** + * Helper to remove a dfs file from the minicluster DFS + * + * @param miniCluster reference to the Minicluster where the file should be deleted + * @param fileName pathname of the file to be deleted + * @throws IOException + */ + static public void deleteFile(MiniCluster miniCluster, String fileName) + throws IOException { + FileSystem fs = miniCluster.getFileSystem(); + fs.delete(new Path(fileName), true); + } +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/NoExitSecurityManager.java hcatalog/core/src/test/java/org/apache/hcatalog/NoExitSecurityManager.java new file mode 100644 index 0000000..31d32be --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/NoExitSecurityManager.java @@ -0,0 +1,45 @@ +/** + * 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.hcatalog; + +import java.security.Permission; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.NoExitSecurityManager} instead + */ +public class NoExitSecurityManager extends SecurityManager { + + @Override + public void checkPermission(Permission perm) { + // allow anything. + } + + @Override + public void checkPermission(Permission perm, Object context) { + // allow anything. + } + + @Override + public void checkExit(int status) { + + super.checkExit(status); + throw new ExitException(status); + } +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/cli/DummyStorageHandler.java hcatalog/core/src/test/java/org/apache/hcatalog/cli/DummyStorageHandler.java new file mode 100644 index 0000000..8d6bffc --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/cli/DummyStorageHandler.java @@ -0,0 +1,300 @@ +/** + * 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.hcatalog.cli; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.metastore.HiveMetaHook; +import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.hadoop.hive.ql.io.HiveOutputFormat; +import org.apache.hadoop.hive.ql.metadata.AuthorizationException; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.metadata.Partition; +import org.apache.hadoop.hive.ql.plan.TableDesc; +import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider; +import org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider; +import org.apache.hadoop.hive.ql.security.authorization.Privilege; +import org.apache.hadoop.hive.serde2.SerDe; +import org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.io.WritableComparable; +import org.apache.hadoop.mapred.InputFormat; +import org.apache.hadoop.mapred.InputSplit; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.OutputFormat; +import org.apache.hadoop.mapred.RecordReader; +import org.apache.hadoop.mapred.RecordWriter; +import org.apache.hadoop.mapred.Reporter; +import org.apache.hadoop.util.Progressable; +import org.apache.hcatalog.data.HCatRecord; +import org.apache.hcatalog.mapreduce.HCatStorageHandler; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.cli.DummyStorageHandler} instead + */ +class DummyStorageHandler extends HCatStorageHandler { + + @Override + public Configuration getConf() { + return null; + } + + @Override + public void setConf(Configuration conf) { + } + + @Override + public Class getInputFormatClass() { + return DummyInputFormat.class; + } + + @Override + public Class getOutputFormatClass() { + return DummyOutputFormat.class; + } + + @Override + public Class getSerDeClass() { + return ColumnarSerDe.class; + } + + @Override + public HiveMetaHook getMetaHook() { + return null; + } + + @Override + public void configureInputJobProperties(TableDesc tableDesc, Map jobProperties) { + } + + @Override + public void configureOutputJobProperties(TableDesc tableDesc, Map jobProperties) { + } + + @Override + public HiveAuthorizationProvider getAuthorizationProvider() + throws HiveException { + return new DummyAuthProvider(); + } + @Override + public void configureJobConf(TableDesc tableDesc, JobConf jobConf) { + //do nothing by default + //EK: added the same (no-op) implementation as in + // org.apache.hive.hcatalog.DefaultStorageHandler (hive 0.12) + // this is needed to get 0.11 API compat layer to work + // see HIVE-4896 + } + + private class DummyAuthProvider implements HiveAuthorizationProvider { + + @Override + public Configuration getConf() { + return null; + } + + /* @param conf + * @see org.apache.hadoop.conf.Configurable#setConf(org.apache.hadoop.conf.Configuration) + */ + @Override + public void setConf(Configuration conf) { + } + + /* @param conf + /* @throws HiveException + * @see org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider#init(org.apache.hadoop.conf.Configuration) + */ + @Override + public void init(Configuration conf) throws HiveException { + } + + /* @return HiveAuthenticationProvider + * @see org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider#getAuthenticator() + */ + @Override + public HiveAuthenticationProvider getAuthenticator() { + return null; + } + + /* @param authenticator + * @see org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider#setAuthenticator(org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider) + */ + @Override + public void setAuthenticator(HiveAuthenticationProvider authenticator) { + } + + /* @param readRequiredPriv + /* @param writeRequiredPriv + /* @throws HiveException + /* @throws AuthorizationException + * @see org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider#authorize(org.apache.hadoop.hive.ql.security.authorization.Privilege[], org.apache.hadoop.hive.ql.security.authorization.Privilege[]) + */ + @Override + public void authorize(Privilege[] readRequiredPriv, + Privilege[] writeRequiredPriv) throws HiveException, + AuthorizationException { + } + + /* @param db + /* @param readRequiredPriv + /* @param writeRequiredPriv + /* @throws HiveException + /* @throws AuthorizationException + * @see org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider#authorize(org.apache.hadoop.hive.metastore.api.Database, org.apache.hadoop.hive.ql.security.authorization.Privilege[], org.apache.hadoop.hive.ql.security.authorization.Privilege[]) + */ + @Override + public void authorize(Database db, Privilege[] readRequiredPriv, + Privilege[] writeRequiredPriv) throws HiveException, + AuthorizationException { + } + + /* @param table + /* @param readRequiredPriv + /* @param writeRequiredPriv + /* @throws HiveException + /* @throws AuthorizationException + * @see org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider#authorize(org.apache.hadoop.hive.ql.metadata.Table, org.apache.hadoop.hive.ql.security.authorization.Privilege[], org.apache.hadoop.hive.ql.security.authorization.Privilege[]) + */ + @Override + public void authorize(org.apache.hadoop.hive.ql.metadata.Table table, Privilege[] readRequiredPriv, + Privilege[] writeRequiredPriv) throws HiveException, + AuthorizationException { + } + + /* @param part + /* @param readRequiredPriv + /* @param writeRequiredPriv + /* @throws HiveException + /* @throws AuthorizationException + * @see org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider#authorize(org.apache.hadoop.hive.ql.metadata.Partition, org.apache.hadoop.hive.ql.security.authorization.Privilege[], org.apache.hadoop.hive.ql.security.authorization.Privilege[]) + */ + @Override + public void authorize(Partition part, Privilege[] readRequiredPriv, + Privilege[] writeRequiredPriv) throws HiveException, + AuthorizationException { + } + + /* @param table + /* @param part + /* @param columns + /* @param readRequiredPriv + /* @param writeRequiredPriv + /* @throws HiveException + /* @throws AuthorizationException + * @see org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider#authorize(org.apache.hadoop.hive.ql.metadata.Table, org.apache.hadoop.hive.ql.metadata.Partition, java.util.List, org.apache.hadoop.hive.ql.security.authorization.Privilege[], org.apache.hadoop.hive.ql.security.authorization.Privilege[]) + */ + @Override + public void authorize(org.apache.hadoop.hive.ql.metadata.Table table, Partition part, List columns, + Privilege[] readRequiredPriv, Privilege[] writeRequiredPriv) + throws HiveException, AuthorizationException { + } + + } + + /** + * The Class DummyInputFormat is a dummy implementation of the old hadoop + * mapred.InputFormat required by HiveStorageHandler. + */ + class DummyInputFormat implements + InputFormat { + + /* + * @see + * org.apache.hadoop.mapred.InputFormat#getRecordReader(org.apache.hadoop + * .mapred.InputSplit, org.apache.hadoop.mapred.JobConf, + * org.apache.hadoop.mapred.Reporter) + */ + @Override + public RecordReader getRecordReader( + InputSplit split, JobConf jobconf, Reporter reporter) + throws IOException { + throw new IOException("This operation is not supported."); + } + + /* + * @see + * org.apache.hadoop.mapred.InputFormat#getSplits(org.apache.hadoop. + * mapred .JobConf, int) + */ + @Override + public InputSplit[] getSplits(JobConf jobconf, int number) + throws IOException { + throw new IOException("This operation is not supported."); + } + } + + /** + * The Class DummyOutputFormat is a dummy implementation of the old hadoop + * mapred.OutputFormat and HiveOutputFormat required by HiveStorageHandler. + */ + class DummyOutputFormat implements + OutputFormat, HCatRecord>, + HiveOutputFormat, HCatRecord> { + + /* + * @see + * org.apache.hadoop.mapred.OutputFormat#checkOutputSpecs(org.apache + * .hadoop .fs.FileSystem, org.apache.hadoop.mapred.JobConf) + */ + @Override + public void checkOutputSpecs(FileSystem fs, JobConf jobconf) + throws IOException { + throw new IOException("This operation is not supported."); + + } + + /* + * @see + * org.apache.hadoop.mapred.OutputFormat#getRecordWriter(org.apache. + * hadoop .fs.FileSystem, org.apache.hadoop.mapred.JobConf, + * java.lang.String, org.apache.hadoop.util.Progressable) + */ + @Override + public RecordWriter, HCatRecord> getRecordWriter( + FileSystem fs, JobConf jobconf, String str, + Progressable progress) throws IOException { + throw new IOException("This operation is not supported."); + } + + /* + * @see + * org.apache.hadoop.hive.ql.io.HiveOutputFormat#getHiveRecordWriter(org + * .apache.hadoop.mapred.JobConf, org.apache.hadoop.fs.Path, + * java.lang.Class, boolean, java.util.Properties, + * org.apache.hadoop.util.Progressable) + */ + @Override + public org.apache.hadoop.hive.ql.exec.FileSinkOperator.RecordWriter getHiveRecordWriter( + JobConf jc, Path finalOutPath, + Class valueClass, boolean isCompressed, + Properties tableProperties, Progressable progress) + throws IOException { + throw new IOException("This operation is not supported."); + } + + } + +} + + diff --git hcatalog/core/src/test/java/org/apache/hcatalog/cli/TestEximSemanticAnalysis.java.broken hcatalog/core/src/test/java/org/apache/hcatalog/cli/TestEximSemanticAnalysis.java.broken new file mode 100644 index 0000000..506a40e --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/cli/TestEximSemanticAnalysis.java.broken @@ -0,0 +1,175 @@ +/** + * 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.hcatalog.cli; + +import java.io.IOException; +import java.net.URI; + +import junit.framework.TestCase; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hive.cli.CliSessionState; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.metastore.Warehouse; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hcatalog.MiniCluster; +import org.apache.hcatalog.cli.SemanticAnalysis.HCatSemanticAnalyzer; +import org.apache.hcatalog.common.HCatConstants; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public class TestEximSemanticAnalysis extends TestCase { + + private final MiniCluster cluster = MiniCluster.buildCluster(); + private HiveConf hcatConf; + private HCatDriver hcatDriver; + private Warehouse wh; + private static final Logger LOG = LoggerFactory.getLogger(TestEximSemanticAnalysis.class); + + @Override + protected void setUp() throws Exception { + + hcatConf = new HiveConf(this.getClass()); + hcatConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); + hcatConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); + hcatConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); + hcatConf.set(ConfVars.SEMANTIC_ANALYZER_HOOK.varname, HCatSemanticAnalyzer.class.getName()); + hcatConf.set("fs.pfile.impl", "org.apache.hadoop.fs.ProxyLocalFileSystem"); + URI fsuri = cluster.getFileSystem().getUri(); + Path whPath = new Path(fsuri.getScheme(), fsuri.getAuthority(), "/user/hive/warehouse"); + hcatConf.set(HiveConf.ConfVars.HADOOPFS.varname, fsuri.toString()); + hcatConf.set(ConfVars.METASTOREWAREHOUSE.varname, whPath.toString()); + wh = new Warehouse(hcatConf); + SessionState.start(new CliSessionState(hcatConf)); + + hcatDriver = new HCatDriver(); + } + + @Override + protected void tearDown() throws Exception { + } + + public void testExportPerms() throws IOException, MetaException, HiveException { + + hcatDriver.run("drop table junit_sem_analysis"); + CommandProcessorResponse response = hcatDriver + .run("create table junit_sem_analysis (a int) partitioned by (b string) stored as RCFILE"); + assertEquals(0, response.getResponseCode()); + Path whPath = wh.getTablePath(Hive.get(hcatConf).getDatabase("default"), "junit_sem_analysis"); + cluster.getFileSystem().setPermission(whPath, FsPermission.valueOf("-rwxrwx-wx")); + cluster.getFileSystem().setOwner(whPath, "nosuchuser", "nosuchgroup"); + + Runtime.getRuntime().exec("rm -rf /tmp/hcat"); + response = hcatDriver + .run("export table junit_sem_analysis to 'pfile://local:9080/tmp/hcat/exports/junit_sem_analysis'"); + + assertEquals(10, response.getResponseCode()); + assertTrue("Permission denied expected : "+response.getErrorMessage(), + response.getErrorMessage().startsWith( + "FAILED: Error in semantic analysis: org.apache.hcatalog.common.HCatException : 3000 : Permission denied")); + Runtime.getRuntime().exec("rm -rf /tmp/hcat"); + response = hcatDriver.run("drop table junit_sem_analysis"); + if (response.getResponseCode() != 0) { + LOG.error(response.getErrorMessage()); + fail("Drop table failed"); + } + } + + public void testImportPerms() throws IOException, MetaException, HiveException { + + hcatDriver.run("drop table junit_sem_analysis"); + CommandProcessorResponse response = hcatDriver + .run("create table junit_sem_analysis (a int) partitioned by (b string) stored as RCFILE"); + assertEquals(0, response.getResponseCode()); + Runtime.getRuntime().exec("rm -rf /tmp/hcat"); + response = hcatDriver + .run("export table junit_sem_analysis to 'pfile://local:9080/tmp/hcat/exports/junit_sem_analysis'"); + assertEquals(0, response.getResponseCode()); + response = hcatDriver.run("drop table junit_sem_analysis"); + assertEquals(0, response.getResponseCode()); + response = hcatDriver + .run("create table junit_sem_analysis (a int) partitioned by (b string) stored as RCFILE"); + assertEquals(0, response.getResponseCode()); + Path whPath = wh.getTablePath(Hive.get(hcatConf).getDatabase("default"), "junit_sem_analysis"); + cluster.getFileSystem().setPermission(whPath, FsPermission.valueOf("-rwxrwxr-x")); + cluster.getFileSystem().setOwner(whPath, "nosuchuser", "nosuchgroup"); + + response = hcatDriver + .run("import table junit_sem_analysis from 'pfile://local:9080/tmp/hcat/exports/junit_sem_analysis'"); + + assertEquals(10, response.getResponseCode()); + assertTrue( + "Permission denied expected: "+response.getErrorMessage() , + response.getErrorMessage().startsWith( + "FAILED: Error in semantic analysis: org.apache.hcatalog.common.HCatException : 3000 : Permission denied")); + Runtime.getRuntime().exec("rm -rf /tmp/hcat"); + + cluster.getFileSystem().setPermission(whPath, FsPermission.valueOf("-rwxrwxrwx")); + response = hcatDriver.run("drop table junit_sem_analysis"); + if (response.getResponseCode() != 0) { + LOG.error(response.getErrorMessage()); + fail("Drop table failed"); + } + } + + public void testImportSetPermsGroup() throws IOException, MetaException, HiveException { + + hcatDriver.run("drop table junit_sem_analysis"); + hcatDriver.run("drop table junit_sem_analysis_imported"); + CommandProcessorResponse response = hcatDriver + .run("create table junit_sem_analysis (a int) partitioned by (b string) stored as RCFILE"); + assertEquals(0, response.getResponseCode()); + Runtime.getRuntime().exec("rm -rf /tmp/hcat"); + response = hcatDriver + .run("export table junit_sem_analysis to 'pfile://local:9080/tmp/hcat/exports/junit_sem_analysis'"); + assertEquals(0, response.getResponseCode()); + response = hcatDriver.run("drop table junit_sem_analysis"); + assertEquals(0, response.getResponseCode()); + + hcatConf.set(HCatConstants.HCAT_PERMS, "-rwxrw-r--"); + hcatConf.set(HCatConstants.HCAT_GROUP, "nosuchgroup"); + + response = hcatDriver + .run("import table junit_sem_analysis_imported from 'pfile://local:9080/tmp/hcat/exports/junit_sem_analysis'"); + assertEquals(0, response.getResponseCode()); + + Path whPath = wh.getTablePath(Hive.get(hcatConf).getDatabase("default"), "junit_sem_analysis_imported"); + assertEquals(FsPermission.valueOf("-rwxrw-r--"), cluster.getFileSystem().getFileStatus(whPath).getPermission()); + assertEquals("nosuchgroup", cluster.getFileSystem().getFileStatus(whPath).getGroup()); + + Runtime.getRuntime().exec("rm -rf /tmp/hcat"); + + response = hcatDriver.run("drop table junit_sem_analysis_imported"); + if (response.getResponseCode() != 0) { + LOG.error(response.getErrorMessage()); + fail("Drop table failed"); + } + } + + +} + diff --git hcatalog/core/src/test/java/org/apache/hcatalog/cli/TestPermsGrp.java hcatalog/core/src/test/java/org/apache/hcatalog/cli/TestPermsGrp.java new file mode 100644 index 0000000..1ec1e3a --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/cli/TestPermsGrp.java @@ -0,0 +1,233 @@ +/** + * 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.hcatalog.cli; + +import java.io.FileNotFoundException; +import java.util.ArrayList; + +import junit.framework.TestCase; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.MetaStoreUtils; +import org.apache.hadoop.hive.metastore.Warehouse; +import org.apache.hadoop.hive.metastore.api.AlreadyExistsException; +import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.InvalidObjectException; +import org.apache.hadoop.hive.metastore.api.InvalidOperationException; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hadoop.hive.metastore.api.SerDeInfo; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.metastore.api.Type; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.serde.serdeConstants; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hcatalog.ExitException; +import org.apache.hcatalog.NoExitSecurityManager; + +import org.apache.hcatalog.cli.SemanticAnalysis.HCatSemanticAnalyzer; +import org.apache.hcatalog.common.HCatConstants; +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.cli.TestPermsGrp} instead + */ +public class TestPermsGrp extends TestCase { + + private boolean isServerRunning = false; + private static final int msPort = 20101; + private HiveConf hcatConf; + private Warehouse clientWH; + private HiveMetaStoreClient msc; + private static final Logger LOG = LoggerFactory.getLogger(TestPermsGrp.class); + + @Override + protected void tearDown() throws Exception { + System.setSecurityManager(securityManager); + } + + @Override + protected void setUp() throws Exception { + + if (isServerRunning) { + return; + } + + MetaStoreUtils.startMetaStore(msPort, ShimLoader.getHadoopThriftAuthBridge()); + + isServerRunning = true; + + securityManager = System.getSecurityManager(); + System.setSecurityManager(new NoExitSecurityManager()); + + hcatConf = new HiveConf(this.getClass()); + hcatConf.set("hive.metastore.local", "false"); + hcatConf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://127.0.0.1:" + msPort); + hcatConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3); + hcatConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTFAILURERETRIES, 3); + + hcatConf.set(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK.varname, HCatSemanticAnalyzer.class.getName()); + hcatConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); + hcatConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); + hcatConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); + clientWH = new Warehouse(hcatConf); + msc = new HiveMetaStoreClient(hcatConf, null); + System.setProperty(HiveConf.ConfVars.PREEXECHOOKS.varname, " "); + System.setProperty(HiveConf.ConfVars.POSTEXECHOOKS.varname, " "); + } + + + public void testCustomPerms() throws Exception { + + String dbName = MetaStoreUtils.DEFAULT_DATABASE_NAME; + String tblName = "simptbl"; + String typeName = "Person"; + + try { + + // Lets first test for default permissions, this is the case when user specified nothing. + Table tbl = getTable(dbName, tblName, typeName); + msc.createTable(tbl); + Database db = Hive.get(hcatConf).getDatabase(dbName); + Path dfsPath = clientWH.getTablePath(db, tblName); + cleanupTbl(dbName, tblName, typeName); + + // Next user did specify perms. + try { + HCatCli.main(new String[]{"-e", "create table simptbl (name string) stored as RCFILE", "-p", "rwx-wx---"}); + } catch (Exception e) { + assertTrue(e instanceof ExitException); + assertEquals(((ExitException) e).getStatus(), 0); + } + dfsPath = clientWH.getTablePath(db, tblName); + assertTrue(dfsPath.getFileSystem(hcatConf).getFileStatus(dfsPath).getPermission().equals(FsPermission.valueOf("drwx-wx---"))); + + cleanupTbl(dbName, tblName, typeName); + + // User specified perms in invalid format. + hcatConf.set(HCatConstants.HCAT_PERMS, "rwx"); + // make sure create table fails. + try { + HCatCli.main(new String[]{"-e", "create table simptbl (name string) stored as RCFILE", "-p", "rwx"}); + assert false; + } catch (Exception me) { + assertTrue(me instanceof ExitException); + } + // No physical dir gets created. + dfsPath = clientWH.getTablePath(db, tblName); + try { + dfsPath.getFileSystem(hcatConf).getFileStatus(dfsPath); + assert false; + } catch (Exception fnfe) { + assertTrue(fnfe instanceof FileNotFoundException); + } + + // And no metadata gets created. + try { + msc.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, tblName); + assert false; + } catch (Exception e) { + assertTrue(e instanceof NoSuchObjectException); + assertEquals("default.simptbl table not found", e.getMessage()); + } + + // test for invalid group name + hcatConf.set(HCatConstants.HCAT_PERMS, "drw-rw-rw-"); + hcatConf.set(HCatConstants.HCAT_GROUP, "THIS_CANNOT_BE_A_VALID_GRP_NAME_EVER"); + + try { + // create table must fail. + HCatCli.main(new String[]{"-e", "create table simptbl (name string) stored as RCFILE", "-p", "rw-rw-rw-", "-g", "THIS_CANNOT_BE_A_VALID_GRP_NAME_EVER"}); + assert false; + } catch (Exception me) { + assertTrue(me instanceof SecurityException); + } + + try { + // no metadata should get created. + msc.getTable(dbName, tblName); + assert false; + } catch (Exception e) { + assertTrue(e instanceof NoSuchObjectException); + assertEquals("default.simptbl table not found", e.getMessage()); + } + try { + // neither dir should get created. + dfsPath.getFileSystem(hcatConf).getFileStatus(dfsPath); + assert false; + } catch (Exception e) { + assertTrue(e instanceof FileNotFoundException); + } + + } catch (Exception e) { + LOG.error("testCustomPerms failed.", e); + throw e; + } + } + + private void silentDropDatabase(String dbName) throws MetaException, TException { + try { + for (String tableName : msc.getTables(dbName, "*")) { + msc.dropTable(dbName, tableName); + } + + } catch (NoSuchObjectException e) { + } + } + + private void cleanupTbl(String dbName, String tblName, String typeName) throws NoSuchObjectException, MetaException, TException, InvalidOperationException { + + msc.dropTable(dbName, tblName); + msc.dropType(typeName); + } + + private Table getTable(String dbName, String tblName, String typeName) throws NoSuchObjectException, MetaException, TException, AlreadyExistsException, InvalidObjectException { + + msc.dropTable(dbName, tblName); + silentDropDatabase(dbName); + + + msc.dropType(typeName); + Type typ1 = new Type(); + typ1.setName(typeName); + typ1.setFields(new ArrayList(1)); + typ1.getFields().add(new FieldSchema("name", serdeConstants.STRING_TYPE_NAME, "")); + msc.createType(typ1); + + Table tbl = new Table(); + tbl.setDbName(dbName); + tbl.setTableName(tblName); + StorageDescriptor sd = new StorageDescriptor(); + tbl.setSd(sd); + sd.setCols(typ1.getFields()); + + sd.setSerdeInfo(new SerDeInfo()); + return tbl; + } + + + private SecurityManager securityManager; + +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/cli/TestSemanticAnalysis.java hcatalog/core/src/test/java/org/apache/hcatalog/cli/TestSemanticAnalysis.java new file mode 100644 index 0000000..c7dcd40 --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/cli/TestSemanticAnalysis.java @@ -0,0 +1,421 @@ +/** + * 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.hcatalog.cli; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.hive.cli.CliSessionState; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.MetaStoreUtils; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.ql.CommandNeedRetryException; +import org.apache.hadoop.hive.ql.Driver; +import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat; +import org.apache.hadoop.hive.ql.io.RCFileInputFormat; +import org.apache.hadoop.hive.ql.io.RCFileOutputFormat; +import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hadoop.mapred.TextInputFormat; +import org.apache.hcatalog.cli.SemanticAnalysis.HCatSemanticAnalyzer; +import org.apache.hcatalog.mapreduce.HCatBaseTest; +import org.apache.thrift.TException; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.cli.TestSemanticAnalysis} instead + */ +public class TestSemanticAnalysis extends HCatBaseTest { + + private static final Logger LOG = LoggerFactory.getLogger(TestSemanticAnalysis.class); + private static final String TBL_NAME = "junit_sem_analysis"; + + private Driver hcatDriver = null; + private String query; + + @Before + public void setUpHCatDriver() throws IOException { + if (hcatDriver == null) { + HiveConf hcatConf = new HiveConf(hiveConf); + hcatConf.set(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK.varname, + HCatSemanticAnalyzer.class.getName()); + hcatDriver = new Driver(hcatConf); + SessionState.start(new CliSessionState(hcatConf)); + } + } + + @Test + public void testDescDB() throws CommandNeedRetryException, IOException { + hcatDriver.run("drop database mydb cascade"); + assertEquals(0, hcatDriver.run("create database mydb").getResponseCode()); + CommandProcessorResponse resp = hcatDriver.run("describe database mydb"); + assertEquals(0, resp.getResponseCode()); + ArrayList result = new ArrayList(); + hcatDriver.getResults(result); + assertTrue(result.get(0).contains("mydb.db")); + hcatDriver.run("drop database mydb cascade"); + } + + @Test + public void testCreateTblWithLowerCasePartNames() throws CommandNeedRetryException, MetaException, TException, NoSuchObjectException { + driver.run("drop table junit_sem_analysis"); + CommandProcessorResponse resp = driver.run("create table junit_sem_analysis (a int) partitioned by (B string) stored as TEXTFILE"); + assertEquals(resp.getResponseCode(), 0); + assertEquals(null, resp.getErrorMessage()); + Table tbl = client.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, TBL_NAME); + assertEquals("Partition key name case problem", "b", tbl.getPartitionKeys().get(0).getName()); + driver.run("drop table junit_sem_analysis"); + } + + @Test + public void testAlterTblFFpart() throws MetaException, TException, NoSuchObjectException, CommandNeedRetryException { + + driver.run("drop table junit_sem_analysis"); + driver.run("create table junit_sem_analysis (a int) partitioned by (b string) stored as TEXTFILE"); + driver.run("alter table junit_sem_analysis add partition (b='2010-10-10')"); + hcatDriver.run("alter table junit_sem_analysis partition (b='2010-10-10') set fileformat RCFILE"); + + Table tbl = client.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, TBL_NAME); + assertEquals(TextInputFormat.class.getName(), tbl.getSd().getInputFormat()); + assertEquals(HiveIgnoreKeyTextOutputFormat.class.getName(), tbl.getSd().getOutputFormat()); + + List partVals = new ArrayList(1); + partVals.add("2010-10-10"); + Partition part = client.getPartition(MetaStoreUtils.DEFAULT_DATABASE_NAME, TBL_NAME, partVals); + + assertEquals(RCFileInputFormat.class.getName(), part.getSd().getInputFormat()); + assertEquals(RCFileOutputFormat.class.getName(), part.getSd().getOutputFormat()); + + hcatDriver.run("drop table junit_sem_analysis"); + } + + @Test + public void testUsNonExistentDB() throws CommandNeedRetryException { + CommandProcessorResponse resp = hcatDriver.run("use no_such_db"); + assertEquals(1, resp.getResponseCode()); + } + + @Test + public void testDatabaseOperations() throws MetaException, CommandNeedRetryException { + + List dbs = client.getAllDatabases(); + String testDb1 = "testdatabaseoperatons1"; + String testDb2 = "testdatabaseoperatons2"; + + if (dbs.contains(testDb1.toLowerCase())) { + assertEquals(0, hcatDriver.run("drop database " + testDb1).getResponseCode()); + } + + if (dbs.contains(testDb2.toLowerCase())) { + assertEquals(0, hcatDriver.run("drop database " + testDb2).getResponseCode()); + } + + assertEquals(0, hcatDriver.run("create database " + testDb1).getResponseCode()); + assertTrue(client.getAllDatabases().contains(testDb1)); + assertEquals(0, hcatDriver.run("create database if not exists " + testDb1).getResponseCode()); + assertTrue(client.getAllDatabases().contains(testDb1)); + assertEquals(0, hcatDriver.run("create database if not exists " + testDb2).getResponseCode()); + assertTrue(client.getAllDatabases().contains(testDb2)); + + assertEquals(0, hcatDriver.run("drop database " + testDb1).getResponseCode()); + assertEquals(0, hcatDriver.run("drop database " + testDb2).getResponseCode()); + assertFalse(client.getAllDatabases().contains(testDb1)); + assertFalse(client.getAllDatabases().contains(testDb2)); + } + + @Test + public void testCreateTableIfNotExists() throws MetaException, TException, NoSuchObjectException, CommandNeedRetryException { + + hcatDriver.run("drop table " + TBL_NAME); + hcatDriver.run("create table junit_sem_analysis (a int) stored as RCFILE"); + Table tbl = client.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, TBL_NAME); + List cols = tbl.getSd().getCols(); + assertEquals(1, cols.size()); + assertTrue(cols.get(0).equals(new FieldSchema("a", "int", null))); + assertEquals(RCFileInputFormat.class.getName(), tbl.getSd().getInputFormat()); + assertEquals(RCFileOutputFormat.class.getName(), tbl.getSd().getOutputFormat()); + + CommandProcessorResponse resp = hcatDriver.run("create table if not exists junit_sem_analysis (a int) stored as RCFILE"); + assertEquals(0, resp.getResponseCode()); + assertNull(resp.getErrorMessage()); + tbl = client.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, TBL_NAME); + cols = tbl.getSd().getCols(); + assertEquals(1, cols.size()); + assertTrue(cols.get(0).equals(new FieldSchema("a", "int", null))); + assertEquals(RCFileInputFormat.class.getName(), tbl.getSd().getInputFormat()); + assertEquals(RCFileOutputFormat.class.getName(), tbl.getSd().getOutputFormat()); + + hcatDriver.run("drop table junit_sem_analysis"); + } + + @Test + public void testAlterTblTouch() throws CommandNeedRetryException { + + hcatDriver.run("drop table junit_sem_analysis"); + hcatDriver.run("create table junit_sem_analysis (a int) partitioned by (b string) stored as RCFILE"); + CommandProcessorResponse response = hcatDriver.run("alter table junit_sem_analysis touch"); + assertEquals(0, response.getResponseCode()); + + hcatDriver.run("alter table junit_sem_analysis touch partition (b='12')"); + assertEquals(0, response.getResponseCode()); + + hcatDriver.run("drop table junit_sem_analysis"); + } + + @Test + public void testChangeColumns() throws CommandNeedRetryException { + hcatDriver.run("drop table junit_sem_analysis"); + hcatDriver.run("create table junit_sem_analysis (a int, c string) partitioned by (b string) stored as RCFILE"); + CommandProcessorResponse response = hcatDriver.run("alter table junit_sem_analysis change a a1 int"); + assertEquals(0, response.getResponseCode()); + + response = hcatDriver.run("alter table junit_sem_analysis change a1 a string"); + assertEquals(0, response.getResponseCode()); + + response = hcatDriver.run("alter table junit_sem_analysis change a a int after c"); + assertEquals(0, response.getResponseCode()); + hcatDriver.run("drop table junit_sem_analysis"); + } + + @Test + public void testAddReplaceCols() throws IOException, MetaException, TException, NoSuchObjectException, CommandNeedRetryException { + + hcatDriver.run("drop table junit_sem_analysis"); + hcatDriver.run("create table junit_sem_analysis (a int, c string) partitioned by (b string)"); + CommandProcessorResponse response = hcatDriver.run("alter table junit_sem_analysis replace columns (a1 tinyint)"); + assertEquals(0, response.getResponseCode()); + + response = hcatDriver.run("alter table junit_sem_analysis add columns (d tinyint)"); + assertEquals(0, response.getResponseCode()); + assertNull(response.getErrorMessage()); + + response = hcatDriver.run("describe extended junit_sem_analysis"); + assertEquals(0, response.getResponseCode()); + Table tbl = client.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, TBL_NAME); + List cols = tbl.getSd().getCols(); + assertEquals(2, cols.size()); + assertTrue(cols.get(0).equals(new FieldSchema("a1", "tinyint", null))); + assertTrue(cols.get(1).equals(new FieldSchema("d", "tinyint", null))); + hcatDriver.run("drop table junit_sem_analysis"); + } + + @Test + public void testAlterTblClusteredBy() throws CommandNeedRetryException { + + hcatDriver.run("drop table junit_sem_analysis"); + hcatDriver.run("create table junit_sem_analysis (a int) partitioned by (b string) stored as RCFILE"); + CommandProcessorResponse response = hcatDriver.run("alter table junit_sem_analysis clustered by (a) into 7 buckets"); + assertEquals(0, response.getResponseCode()); + hcatDriver.run("drop table junit_sem_analysis"); + } + + @Test + public void testAlterTableSetFF() throws IOException, MetaException, TException, NoSuchObjectException, CommandNeedRetryException { + + hcatDriver.run("drop table junit_sem_analysis"); + hcatDriver.run("create table junit_sem_analysis (a int) partitioned by (b string) stored as RCFILE"); + + Table tbl = client.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, TBL_NAME); + assertEquals(RCFileInputFormat.class.getName(), tbl.getSd().getInputFormat()); + assertEquals(RCFileOutputFormat.class.getName(), tbl.getSd().getOutputFormat()); + + hcatDriver.run("alter table junit_sem_analysis set fileformat INPUTFORMAT 'org.apache.hadoop.hive.ql.io.RCFileInputFormat' OUTPUTFORMAT " + + "'org.apache.hadoop.hive.ql.io.RCFileOutputFormat' inputdriver 'mydriver' outputdriver 'yourdriver'"); + hcatDriver.run("desc extended junit_sem_analysis"); + + tbl = client.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, TBL_NAME); + assertEquals(RCFileInputFormat.class.getName(), tbl.getSd().getInputFormat()); + assertEquals(RCFileOutputFormat.class.getName(), tbl.getSd().getOutputFormat()); + + hcatDriver.run("drop table junit_sem_analysis"); + } + + @Test + public void testAddPartFail() throws CommandNeedRetryException { + + driver.run("drop table junit_sem_analysis"); + driver.run("create table junit_sem_analysis (a int) partitioned by (b string) stored as RCFILE"); + CommandProcessorResponse response = hcatDriver.run("alter table junit_sem_analysis add partition (b='2') location 'README.txt'"); + assertEquals(0, response.getResponseCode()); + driver.run("drop table junit_sem_analysis"); + } + + @Test + public void testAddPartPass() throws IOException, CommandNeedRetryException { + + hcatDriver.run("drop table junit_sem_analysis"); + hcatDriver.run("create table junit_sem_analysis (a int) partitioned by (b string) stored as RCFILE"); + CommandProcessorResponse response = hcatDriver.run("alter table junit_sem_analysis add partition (b='2') location '" + TEST_DATA_DIR + "'"); + assertEquals(0, response.getResponseCode()); + assertNull(response.getErrorMessage()); + hcatDriver.run("drop table junit_sem_analysis"); + } + + @Test + public void testCTAS() throws CommandNeedRetryException { + hcatDriver.run("drop table junit_sem_analysis"); + query = "create table junit_sem_analysis (a int) as select * from tbl2"; + CommandProcessorResponse response = hcatDriver.run(query); + assertEquals(40000, response.getResponseCode()); + assertTrue(response.getErrorMessage().contains("FAILED: SemanticException Operation not supported. Create table as Select is not a valid operation.")); + hcatDriver.run("drop table junit_sem_analysis"); + } + + @Test + public void testStoredAs() throws CommandNeedRetryException { + hcatDriver.run("drop table junit_sem_analysis"); + query = "create table junit_sem_analysis (a int)"; + CommandProcessorResponse response = hcatDriver.run(query); + assertEquals(0, response.getResponseCode()); + hcatDriver.run("drop table junit_sem_analysis"); + } + + @Test + public void testAddDriverInfo() throws IOException, MetaException, TException, NoSuchObjectException, CommandNeedRetryException { + + hcatDriver.run("drop table junit_sem_analysis"); + query = "create table junit_sem_analysis (a int) partitioned by (b string) stored as " + + "INPUTFORMAT 'org.apache.hadoop.hive.ql.io.RCFileInputFormat' OUTPUTFORMAT " + + "'org.apache.hadoop.hive.ql.io.RCFileOutputFormat' inputdriver 'mydriver' outputdriver 'yourdriver' "; + assertEquals(0, hcatDriver.run(query).getResponseCode()); + + Table tbl = client.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, TBL_NAME); + assertEquals(RCFileInputFormat.class.getName(), tbl.getSd().getInputFormat()); + assertEquals(RCFileOutputFormat.class.getName(), tbl.getSd().getOutputFormat()); + + hcatDriver.run("drop table junit_sem_analysis"); + } + + @Test + public void testInvalidateNonStringPartition() throws IOException, CommandNeedRetryException { + + hcatDriver.run("drop table junit_sem_analysis"); + query = "create table junit_sem_analysis (a int) partitioned by (b int) stored as RCFILE"; + + CommandProcessorResponse response = hcatDriver.run(query); + assertEquals(40000, response.getResponseCode()); + assertEquals("FAILED: SemanticException Operation not supported. HCatalog only supports partition columns of type string. For column: b Found type: int", + response.getErrorMessage()); + + } + + @Test + public void testInvalidateSeqFileStoredAs() throws IOException, CommandNeedRetryException { + + hcatDriver.run("drop table junit_sem_analysis"); + query = "create table junit_sem_analysis (a int) partitioned by (b string) stored as SEQUENCEFILE"; + + CommandProcessorResponse response = hcatDriver.run(query); + assertEquals(0, response.getResponseCode()); + + } + + @Test + public void testInvalidateTextFileStoredAs() throws IOException, CommandNeedRetryException { + + hcatDriver.run("drop table junit_sem_analysis"); + query = "create table junit_sem_analysis (a int) partitioned by (b string) stored as TEXTFILE"; + + CommandProcessorResponse response = hcatDriver.run(query); + assertEquals(0, response.getResponseCode()); + + } + + @Test + public void testInvalidateClusteredBy() throws IOException, CommandNeedRetryException { + + hcatDriver.run("drop table junit_sem_analysis"); + query = "create table junit_sem_analysis (a int) partitioned by (b string) clustered by (a) into 10 buckets stored as TEXTFILE"; + + CommandProcessorResponse response = hcatDriver.run(query); + assertEquals(0, response.getResponseCode()); + } + + @Test + public void testCTLFail() throws IOException, CommandNeedRetryException { + + driver.run("drop table junit_sem_analysis"); + driver.run("drop table like_table"); + query = "create table junit_sem_analysis (a int) partitioned by (b string) stored as RCFILE"; + + driver.run(query); + query = "create table like_table like junit_sem_analysis"; + CommandProcessorResponse response = hcatDriver.run(query); + assertEquals(0, response.getResponseCode()); + } + + @Test + public void testCTLPass() throws IOException, MetaException, TException, NoSuchObjectException, CommandNeedRetryException { + + try { + hcatDriver.run("drop table junit_sem_analysis"); + } catch (Exception e) { + LOG.error("Error in drop table.", e); + } + query = "create table junit_sem_analysis (a int) partitioned by (b string) stored as RCFILE"; + + hcatDriver.run(query); + String likeTbl = "like_table"; + hcatDriver.run("drop table " + likeTbl); + query = "create table like_table like junit_sem_analysis"; + CommandProcessorResponse resp = hcatDriver.run(query); + assertEquals(0, resp.getResponseCode()); +// Table tbl = client.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, likeTbl); +// assertEquals(likeTbl,tbl.getTableName()); +// List cols = tbl.getSd().getCols(); +// assertEquals(1, cols.size()); +// assertEquals(new FieldSchema("a", "int", null), cols.get(0)); +// assertEquals("org.apache.hadoop.hive.ql.io.RCFileInputFormat",tbl.getSd().getInputFormat()); +// assertEquals("org.apache.hadoop.hive.ql.io.RCFileOutputFormat",tbl.getSd().getOutputFormat()); +// Map tblParams = tbl.getParameters(); +// assertEquals("org.apache.hadoop.hive.hcat.rcfile.RCFileInputStorageDriver", tblParams.get("hcat.isd")); +// assertEquals("org.apache.hadoop.hive.hcat.rcfile.RCFileOutputStorageDriver", tblParams.get("hcat.osd")); +// +// hcatDriver.run("drop table junit_sem_analysis"); +// hcatDriver.run("drop table "+likeTbl); + } + +// This test case currently fails, since add partitions don't inherit anything from tables. + +// public void testAddPartInheritDrivers() throws MetaException, TException, NoSuchObjectException{ +// +// hcatDriver.run("drop table "+TBL_NAME); +// hcatDriver.run("create table junit_sem_analysis (a int) partitioned by (b string) stored as RCFILE"); +// hcatDriver.run("alter table "+TBL_NAME+" add partition (b='2010-10-10')"); +// +// List partVals = new ArrayList(1); +// partVals.add("2010-10-10"); +// +// Map map = client.getPartition(MetaStoreUtils.DEFAULT_DATABASE_NAME, TBL_NAME, partVals).getParameters(); +// assertEquals(map.get(InitializeInput.HOWL_ISD_CLASS), RCFileInputStorageDriver.class.getName()); +// assertEquals(map.get(InitializeInput.HOWL_OSD_CLASS), RCFileOutputStorageDriver.class.getName()); +// } +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/cli/TestStorageHandlerProperties.java.broken hcatalog/core/src/test/java/org/apache/hcatalog/cli/TestStorageHandlerProperties.java.broken new file mode 100644 index 0000000..7612337 --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/cli/TestStorageHandlerProperties.java.broken @@ -0,0 +1,86 @@ +/** + * 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.hcatalog.cli; + +import static org.junit.Assert.assertEquals; + +import org.apache.hadoop.hive.cli.CliSessionState; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.MetaStoreUtils; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.ql.CommandNeedRetryException; +import org.apache.hadoop.hive.ql.Driver; +import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hcatalog.cli.SemanticAnalysis.HCatSemanticAnalyzer; +import org.apache.hcatalog.common.HCatConstants; +import org.apache.thrift.TException; + +import junit.framework.TestCase; + +public class TestStorageHandlerProperties extends TestCase { + + private Driver hcatDriver; + private Driver hiveDriver; + private HiveMetaStoreClient msc; + + protected void setUp() throws Exception { + HiveConf hcatConf = new HiveConf(this.getClass()); + hcatConf.set(ConfVars.PREEXECHOOKS.varname, ""); + hcatConf.set(ConfVars.POSTEXECHOOKS.varname, ""); + hcatConf.set(ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); + + HiveConf hiveConf = new HiveConf(hcatConf,this.getClass()); + hiveDriver = new Driver(hiveConf); + + hcatConf.set(ConfVars.SEMANTIC_ANALYZER_HOOK.varname, HCatSemanticAnalyzer.class.getName()); + hcatDriver = new Driver(hcatConf); + + msc = new HiveMetaStoreClient(hcatConf); + SessionState.start(new CliSessionState(hcatConf)); + } + + public void testTableProperties() throws CommandNeedRetryException, MetaException ,TException, NoSuchObjectException{ + hcatDriver.run("drop table test_table"); + CommandProcessorResponse response = hcatDriver + .run("create table test_table(key int, value string) STORED BY " + + "'org.apache.hcatalog.cli.DummyStorageHandler' "); + + assertEquals(0, response.getResponseCode()); + Table tbl = msc.getTable(MetaStoreUtils.DEFAULT_DATABASE_NAME, "test_table"); + DummyStorageHandler dsh = new DummyStorageHandler(); + assertTrue(tbl.getParameters().containsKey(HCatConstants.HCAT_ISD_CLASS)); + assertTrue(tbl.getParameters().containsKey(HCatConstants.HCAT_OSD_CLASS)); + assertEquals(tbl.getParameters().get(HCatConstants.HCAT_ISD_CLASS), dsh.getInputStorageDriver().getName()); + assertEquals(tbl.getParameters().get(HCatConstants.HCAT_OSD_CLASS), dsh.getOutputStorageDriver().getName()); + } + + /* @throws java.lang.Exception + * @see junit.framework.TestCase#tearDown() + */ + protected void tearDown() throws Exception { + super.tearDown(); + } + +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/cli/TestUseDatabase.java hcatalog/core/src/test/java/org/apache/hcatalog/cli/TestUseDatabase.java new file mode 100644 index 0000000..c2ef9c6 --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/cli/TestUseDatabase.java @@ -0,0 +1,79 @@ +/** + * 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.hcatalog.cli; + +import java.io.IOException; + +import junit.framework.TestCase; + +import org.apache.hadoop.hive.cli.CliSessionState; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.ql.CommandNeedRetryException; +import org.apache.hadoop.hive.ql.Driver; +import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hcatalog.cli.SemanticAnalysis.HCatSemanticAnalyzer; + +/** Unit test for GitHub Howl issue #3 + * @deprecated Use/modify {@link org.apache.hive.hcatalog.cli.TestUseDatabase} instead + */ +public class TestUseDatabase extends TestCase { + + private Driver hcatDriver; + + @Override + protected void setUp() throws Exception { + + HiveConf hcatConf = new HiveConf(this.getClass()); + hcatConf.set(ConfVars.PREEXECHOOKS.varname, ""); + hcatConf.set(ConfVars.POSTEXECHOOKS.varname, ""); + hcatConf.set(ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); + + hcatConf.set(ConfVars.SEMANTIC_ANALYZER_HOOK.varname, HCatSemanticAnalyzer.class.getName()); + hcatDriver = new Driver(hcatConf); + SessionState.start(new CliSessionState(hcatConf)); + } + + String query; + private final String dbName = "testUseDatabase_db"; + private final String tblName = "testUseDatabase_tbl"; + + public void testAlterTablePass() throws IOException, CommandNeedRetryException { + + hcatDriver.run("create database " + dbName); + hcatDriver.run("use " + dbName); + hcatDriver.run("create table " + tblName + " (a int) partitioned by (b string) stored as RCFILE"); + + CommandProcessorResponse response; + + response = hcatDriver.run("alter table " + tblName + " add partition (b='2') location '/tmp'"); + assertEquals(0, response.getResponseCode()); + assertNull(response.getErrorMessage()); + + response = hcatDriver.run("alter table " + tblName + " set fileformat INPUTFORMAT 'org.apache.hadoop.hive.ql.io.RCFileInputFormat' OUTPUTFORMAT " + + "'org.apache.hadoop.hive.ql.io.RCFileOutputFormat' inputdriver 'mydriver' outputdriver 'yourdriver'"); + assertEquals(0, response.getResponseCode()); + assertNull(response.getErrorMessage()); + + hcatDriver.run("drop table " + tblName); + hcatDriver.run("drop database " + dbName); + } + +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/common/TestHCatUtil.java hcatalog/core/src/test/java/org/apache/hcatalog/common/TestHCatUtil.java new file mode 100644 index 0000000..2205b49 --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/common/TestHCatUtil.java @@ -0,0 +1,185 @@ +/** + * 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.hcatalog.common; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import org.apache.hadoop.fs.permission.FsAction; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hive.metastore.TableType; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.Order; +import org.apache.hadoop.hive.metastore.api.SerDeInfo; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.serde.serdeConstants; +import org.apache.hcatalog.data.schema.HCatFieldSchema; +import org.apache.hcatalog.data.schema.HCatSchema; +import org.junit.Assert; +import org.junit.Test; +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.common.TestHCatUtil} instead + */ +public class TestHCatUtil { + + @Test + public void testFsPermissionOperation() { + + HashMap permsCode = new HashMap(); + + for (int i = 0; i < 8; i++) { + for (int j = 0; j < 8; j++) { + for (int k = 0; k < 8; k++) { + StringBuilder sb = new StringBuilder(); + sb.append("0"); + sb.append(i); + sb.append(j); + sb.append(k); + Integer code = (((i * 8) + j) * 8) + k; + String perms = (new FsPermission(Short.decode(sb.toString()))).toString(); + if (permsCode.containsKey(perms)) { + Assert.assertEquals("permissions(" + perms + ") mapped to multiple codes", code, permsCode.get(perms)); + } + permsCode.put(perms, code); + assertFsPermissionTransformationIsGood(perms); + } + } + } + } + + private void assertFsPermissionTransformationIsGood(String perms) { + Assert.assertEquals(perms, FsPermission.valueOf("-" + perms).toString()); + } + + @Test + public void testValidateMorePermissive() { + assertConsistentFsPermissionBehaviour(FsAction.ALL, true, true, true, true, true, true, true, true); + assertConsistentFsPermissionBehaviour(FsAction.READ, false, true, false, true, false, false, false, false); + assertConsistentFsPermissionBehaviour(FsAction.WRITE, false, true, false, false, true, false, false, false); + assertConsistentFsPermissionBehaviour(FsAction.EXECUTE, false, true, true, false, false, false, false, false); + assertConsistentFsPermissionBehaviour(FsAction.READ_EXECUTE, false, true, true, true, false, true, false, false); + assertConsistentFsPermissionBehaviour(FsAction.READ_WRITE, false, true, false, true, true, false, true, false); + assertConsistentFsPermissionBehaviour(FsAction.WRITE_EXECUTE, false, true, true, false, true, false, false, true); + assertConsistentFsPermissionBehaviour(FsAction.NONE, false, true, false, false, false, false, false, false); + } + + + private void assertConsistentFsPermissionBehaviour( + FsAction base, boolean versusAll, boolean versusNone, + boolean versusX, boolean versusR, boolean versusW, + boolean versusRX, boolean versusRW, boolean versusWX) { + + Assert.assertTrue(versusAll == HCatUtil.validateMorePermissive(base, FsAction.ALL)); + Assert.assertTrue(versusX == HCatUtil.validateMorePermissive(base, FsAction.EXECUTE)); + Assert.assertTrue(versusNone == HCatUtil.validateMorePermissive(base, FsAction.NONE)); + Assert.assertTrue(versusR == HCatUtil.validateMorePermissive(base, FsAction.READ)); + Assert.assertTrue(versusRX == HCatUtil.validateMorePermissive(base, FsAction.READ_EXECUTE)); + Assert.assertTrue(versusRW == HCatUtil.validateMorePermissive(base, FsAction.READ_WRITE)); + Assert.assertTrue(versusW == HCatUtil.validateMorePermissive(base, FsAction.WRITE)); + Assert.assertTrue(versusWX == HCatUtil.validateMorePermissive(base, FsAction.WRITE_EXECUTE)); + } + + @Test + public void testExecutePermissionsCheck() { + Assert.assertTrue(HCatUtil.validateExecuteBitPresentIfReadOrWrite(FsAction.ALL)); + Assert.assertTrue(HCatUtil.validateExecuteBitPresentIfReadOrWrite(FsAction.NONE)); + Assert.assertTrue(HCatUtil.validateExecuteBitPresentIfReadOrWrite(FsAction.EXECUTE)); + Assert.assertTrue(HCatUtil.validateExecuteBitPresentIfReadOrWrite(FsAction.READ_EXECUTE)); + Assert.assertTrue(HCatUtil.validateExecuteBitPresentIfReadOrWrite(FsAction.WRITE_EXECUTE)); + + Assert.assertFalse(HCatUtil.validateExecuteBitPresentIfReadOrWrite(FsAction.READ)); + Assert.assertFalse(HCatUtil.validateExecuteBitPresentIfReadOrWrite(FsAction.WRITE)); + Assert.assertFalse(HCatUtil.validateExecuteBitPresentIfReadOrWrite(FsAction.READ_WRITE)); + + } + + @Test + public void testGetTableSchemaWithPtnColsApi() throws IOException { + // Check the schema of a table with one field & no partition keys. + StorageDescriptor sd = new StorageDescriptor( + Lists.newArrayList(new FieldSchema("username", serdeConstants.STRING_TYPE_NAME, null)), + "location", "org.apache.hadoop.mapred.TextInputFormat", + "org.apache.hadoop.mapred.TextOutputFormat", false, -1, new SerDeInfo(), + new ArrayList(), new ArrayList(), new HashMap()); + org.apache.hadoop.hive.metastore.api.Table apiTable = + new org.apache.hadoop.hive.metastore.api.Table("test_tblname", "test_dbname", "test_owner", + 0, 0, 0, sd, new ArrayList(), new HashMap(), + "viewOriginalText", "viewExpandedText", TableType.EXTERNAL_TABLE.name()); + Table table = new Table(apiTable); + + List expectedHCatSchema = + Lists.newArrayList(new HCatFieldSchema("username", HCatFieldSchema.Type.STRING, null)); + + Assert.assertEquals(new HCatSchema(expectedHCatSchema), + HCatUtil.getTableSchemaWithPtnCols(table)); + + // Add a partition key & ensure its reflected in the schema. + List partitionKeys = + Lists.newArrayList(new FieldSchema("dt", serdeConstants.STRING_TYPE_NAME, null)); + table.getTTable().setPartitionKeys(partitionKeys); + expectedHCatSchema.add(new HCatFieldSchema("dt", HCatFieldSchema.Type.STRING, null)); + Assert.assertEquals(new HCatSchema(expectedHCatSchema), + HCatUtil.getTableSchemaWithPtnCols(table)); + } + + /** + * Hive represents tables in two ways: + *

    + *
  • org.apache.hadoop.hive.metastore.api.Table - exactly whats stored in the metastore
  • + *
  • org.apache.hadoop.hive.ql.metadata.Table - adds business logic over api.Table
  • + *
+ * Here we check SerDe-reported fields are included in the table schema. + */ + @Test + public void testGetTableSchemaWithPtnColsSerDeReportedFields() throws IOException { + Map parameters = Maps.newHashMap(); + parameters.put(serdeConstants.SERIALIZATION_CLASS, + "org.apache.hadoop.hive.serde2.thrift.test.IntString"); + parameters.put(serdeConstants.SERIALIZATION_FORMAT, "org.apache.thrift.protocol.TBinaryProtocol"); + + SerDeInfo serDeInfo = new SerDeInfo(null, + "org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer", parameters); + + // StorageDescriptor has an empty list of fields - SerDe will report them. + StorageDescriptor sd = new StorageDescriptor(new ArrayList(), "location", + "org.apache.hadoop.mapred.TextInputFormat", "org.apache.hadoop.mapred.TextOutputFormat", + false, -1, serDeInfo, new ArrayList(), new ArrayList(), + new HashMap()); + + org.apache.hadoop.hive.metastore.api.Table apiTable = + new org.apache.hadoop.hive.metastore.api.Table("test_tblname", "test_dbname", "test_owner", + 0, 0, 0, sd, new ArrayList(), new HashMap(), + "viewOriginalText", "viewExpandedText", TableType.EXTERNAL_TABLE.name()); + Table table = new Table(apiTable); + + List expectedHCatSchema = Lists.newArrayList( + new HCatFieldSchema("myint", HCatFieldSchema.Type.INT, null), + new HCatFieldSchema("mystring", HCatFieldSchema.Type.STRING, null), + new HCatFieldSchema("underscore_int", HCatFieldSchema.Type.INT, null)); + + Assert.assertEquals(new HCatSchema(expectedHCatSchema), + HCatUtil.getTableSchemaWithPtnCols(table)); + } +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/common/TestHiveClientCache.java hcatalog/core/src/test/java/org/apache/hcatalog/common/TestHiveClientCache.java new file mode 100644 index 0000000..eb52080 --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/common/TestHiveClientCache.java @@ -0,0 +1,269 @@ +/** + * 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.hcatalog.common; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStore; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.AlreadyExistsException; +import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.InvalidObjectException; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hadoop.hive.metastore.api.SerDeInfo; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.serde.serdeConstants; +import org.apache.hcatalog.NoExitSecurityManager; +import org.apache.hcatalog.cli.SemanticAnalysis.HCatSemanticAnalyzer; +import org.apache.thrift.TException; +import org.junit.AfterClass; +import org.junit.BeforeClass; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNotSame; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import org.junit.Ignore; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.security.auth.login.LoginException; +import java.io.IOException; +import java.math.BigInteger; +import java.util.ArrayList; +import java.util.List; +import java.util.Random; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.common.TestHiveClientCache} instead + */ +public class TestHiveClientCache { + + private static final Logger LOG = LoggerFactory.getLogger(TestHiveClientCache.class); + final HiveConf hiveConf = new HiveConf(); + + @BeforeClass + public static void setUp() throws Exception { + } + + @AfterClass + public static void tearDown() throws Exception { + } + + @Test + public void testCacheHit() throws IOException, MetaException, LoginException { + + HiveClientCache cache = new HiveClientCache(1000); + HiveMetaStoreClient client = cache.get(hiveConf); + assertNotNull(client); + client.close(); // close shouldn't matter + + // Setting a non important configuration should return the same client only + hiveConf.setIntVar(HiveConf.ConfVars.DYNAMICPARTITIONMAXPARTS, 10); + HiveMetaStoreClient client2 = cache.get(hiveConf); + assertNotNull(client2); + assertEquals(client, client2); + client2.close(); + } + + @Test + public void testCacheMiss() throws IOException, MetaException, LoginException { + HiveClientCache cache = new HiveClientCache(1000); + HiveMetaStoreClient client = cache.get(hiveConf); + assertNotNull(client); + + // Set different uri as it is one of the criteria deciding whether to return the same client or not + hiveConf.setVar(HiveConf.ConfVars.METASTOREURIS, " "); // URIs are checked for string equivalence, even spaces make them different + HiveMetaStoreClient client2 = cache.get(hiveConf); + assertNotNull(client2); + assertNotSame(client, client2); + } + + /** + * Check that a new client is returned for the same configuration after the expiry time. + * Also verify that the expiry time configuration is honoured + */ + @Test + public void testCacheExpiry() throws IOException, MetaException, LoginException, InterruptedException { + HiveClientCache cache = new HiveClientCache(1); + HiveClientCache.CacheableHiveMetaStoreClient client = (HiveClientCache.CacheableHiveMetaStoreClient) cache.get(hiveConf); + assertNotNull(client); + + Thread.sleep(2500); + HiveMetaStoreClient client2 = cache.get(hiveConf); + client.close(); + assertTrue(client.isClosed()); // close() after *expiry time* and *a cache access* should have tore down the client + + assertNotNull(client2); + assertNotSame(client, client2); + } + + /** + * Check that a *new* client is created if asked from different threads even with + * the same hive configuration + * @throws ExecutionException + * @throws InterruptedException + */ + @Test + public void testMultipleThreadAccess() throws ExecutionException, InterruptedException { + final HiveClientCache cache = new HiveClientCache(1000); + + class GetHiveClient implements Callable { + @Override + public HiveMetaStoreClient call() throws IOException, MetaException, LoginException { + return cache.get(hiveConf); + } + } + + ExecutorService executor = Executors.newFixedThreadPool(2); + + Callable worker1 = new GetHiveClient(); + Callable worker2 = new GetHiveClient(); + Future clientFuture1 = executor.submit(worker1); + Future clientFuture2 = executor.submit(worker2); + HiveMetaStoreClient client1 = clientFuture1.get(); + HiveMetaStoreClient client2 = clientFuture2.get(); + assertNotNull(client1); + assertNotNull(client2); + assertNotSame(client1, client2); + } + + @Test + public void testCloseAllClients() throws IOException, MetaException, LoginException { + final HiveClientCache cache = new HiveClientCache(1000); + HiveClientCache.CacheableHiveMetaStoreClient client1 = (HiveClientCache.CacheableHiveMetaStoreClient) cache.get(hiveConf); + hiveConf.setVar(HiveConf.ConfVars.METASTOREURIS, " "); // URIs are checked for string equivalence, even spaces make them different + HiveClientCache.CacheableHiveMetaStoreClient client2 = (HiveClientCache.CacheableHiveMetaStoreClient) cache.get(hiveConf); + cache.closeAllClientsQuietly(); + assertTrue(client1.isClosed()); + assertTrue(client2.isClosed()); + } + + /** + * Test that a long table name actually breaks the HMSC. Subsequently check that isOpen() reflects + * and tells if the client is broken + */ + @Ignore("hangs indefinitely") + @Test + public void testHMSCBreakability() throws IOException, MetaException, LoginException, TException, AlreadyExistsException, + InvalidObjectException, NoSuchObjectException, InterruptedException { + // Setup + LocalMetaServer metaServer = new LocalMetaServer(); + metaServer.start(); + + final HiveClientCache cache = new HiveClientCache(1000); + HiveClientCache.CacheableHiveMetaStoreClient client = + (HiveClientCache.CacheableHiveMetaStoreClient) cache.get(metaServer.getHiveConf()); + + assertTrue(client.isOpen()); + + final String DB_NAME = "test_db"; + final String LONG_TABLE_NAME = "long_table_name_" + new BigInteger(200, new Random()).toString(2); + + try { + client.dropTable(DB_NAME, LONG_TABLE_NAME); + } catch (Exception e) { + } + try { + client.dropDatabase(DB_NAME); + } catch (Exception e) { + } + + client.createDatabase(new Database(DB_NAME, "", null, null)); + + List fields = new ArrayList(); + fields.add(new FieldSchema("colname", serdeConstants.STRING_TYPE_NAME, "")); + Table tbl = new Table(); + tbl.setDbName(DB_NAME); + tbl.setTableName(LONG_TABLE_NAME); + StorageDescriptor sd = new StorageDescriptor(); + sd.setCols(fields); + tbl.setSd(sd); + sd.setSerdeInfo(new SerDeInfo()); + + // Break the client + try { + client.createTable(tbl); + fail("Exception was expected while creating table with long name"); + } catch (Exception e) { + } + + assertFalse(client.isOpen()); + metaServer.shutDown(); + } + + private static class LocalMetaServer implements Runnable { + public final int MS_PORT = 20101; + private final HiveConf hiveConf; + private final SecurityManager securityManager; + public final static int WAIT_TIME_FOR_BOOTUP = 30000; + + public LocalMetaServer() { + securityManager = System.getSecurityManager(); + System.setSecurityManager(new NoExitSecurityManager()); + hiveConf = new HiveConf(TestHiveClientCache.class); + hiveConf.set("hive.metastore.local", "false"); + hiveConf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:" + + MS_PORT); + hiveConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3); + hiveConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTFAILURERETRIES, 3); + hiveConf.set(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK.varname, + HCatSemanticAnalyzer.class.getName()); + hiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, + "false"); + System.setProperty(HiveConf.ConfVars.PREEXECHOOKS.varname, " "); + System.setProperty(HiveConf.ConfVars.POSTEXECHOOKS.varname, " "); + } + + public void start() throws InterruptedException { + Thread thread = new Thread(this); + thread.start(); + Thread.sleep(WAIT_TIME_FOR_BOOTUP); // Wait for the server to bootup + } + + @Override + public void run() { + try { + HiveMetaStore.main(new String[]{"-v", "-p", String.valueOf(MS_PORT)}); + } catch (Throwable t) { + LOG.error("Exiting. Got exception from metastore: ", t); + } + } + + public HiveConf getHiveConf() { + return hiveConf; + } + + public void shutDown() { + System.setSecurityManager(securityManager); + } + } +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/data/HCatDataCheckUtil.java hcatalog/core/src/test/java/org/apache/hcatalog/data/HCatDataCheckUtil.java new file mode 100644 index 0000000..673a038 --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/data/HCatDataCheckUtil.java @@ -0,0 +1,115 @@ +/** + * 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.hcatalog.data; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map.Entry; + +import org.apache.hadoop.hive.cli.CliSessionState; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.CommandNeedRetryException; +import org.apache.hadoop.hive.ql.Driver; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hcatalog.MiniCluster; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Helper class for Other Data Testers + * @deprecated Use/modify {@link org.apache.hive.hcatalog.data.HCatDataCheckUtil} instead + */ +public class HCatDataCheckUtil { + + private static final Logger LOG = LoggerFactory.getLogger(HCatDataCheckUtil.class); + + public static Driver instantiateDriver(MiniCluster cluster) { + HiveConf hiveConf = new HiveConf(HCatDataCheckUtil.class); + for (Entry e : cluster.getProperties().entrySet()) { + hiveConf.set(e.getKey().toString(), e.getValue().toString()); + } + hiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); + + LOG.debug("Hive conf : {}", hiveConf.getAllProperties()); + Driver driver = new Driver(hiveConf); + SessionState.start(new CliSessionState(hiveConf)); + return driver; + } + + public static void generateDataFile(MiniCluster cluster, String fileName) throws IOException { + MiniCluster.deleteFile(cluster, fileName); + String[] input = new String[50]; + for (int i = 0; i < 50; i++) { + input[i] = (i % 5) + "\t" + i + "\t" + "_S" + i + "S_"; + } + MiniCluster.createInputFile(cluster, fileName, input); + } + + public static void createTable(Driver driver, String tableName, String createTableArgs) + throws CommandNeedRetryException, IOException { + String createTable = "create table " + tableName + createTableArgs; + int retCode = driver.run(createTable).getResponseCode(); + if (retCode != 0) { + throw new IOException("Failed to create table. [" + createTable + "], return code from hive driver : [" + retCode + "]"); + } + } + + public static void dropTable(Driver driver, String tablename) throws IOException, CommandNeedRetryException { + driver.run("drop table if exists " + tablename); + } + + public static ArrayList formattedRun(Driver driver, String name, String selectCmd) + throws CommandNeedRetryException, IOException { + driver.run(selectCmd); + ArrayList src_values = new ArrayList(); + driver.getResults(src_values); + LOG.info("{} : {}", name, src_values); + return src_values; + } + + + public static boolean recordsEqual(HCatRecord first, HCatRecord second) { + return (compareRecords(first, second) == 0); + } + + public static int compareRecords(HCatRecord first, HCatRecord second) { + return compareRecordContents(first.getAll(), second.getAll()); + } + + public static int compareRecordContents(List first, List second) { + int mySz = first.size(); + int urSz = second.size(); + if (mySz != urSz) { + return mySz - urSz; + } else { + for (int i = 0; i < first.size(); i++) { + int c = DataType.compare(first.get(i), second.get(i)); + if (c != 0) { + return c; + } + } + return 0; + } + } + + +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/data/TestDefaultHCatRecord.java hcatalog/core/src/test/java/org/apache/hcatalog/data/TestDefaultHCatRecord.java new file mode 100644 index 0000000..799b814 --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/data/TestDefaultHCatRecord.java @@ -0,0 +1,262 @@ +/** + * 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.hcatalog.data; + +import java.io.DataInput; +import java.io.DataInputStream; +import java.io.DataOutput; +import java.io.DataOutputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.data.schema.HCatSchema; +import org.apache.hcatalog.data.schema.HCatSchemaUtils; + +import junit.framework.Assert; +import junit.framework.TestCase; +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.data.TestDefaultHCatRecord} instead + */ +public class TestDefaultHCatRecord extends TestCase { + + public void testRYW() throws IOException { + + File f = new File("binary.dat"); + f.delete(); + f.createNewFile(); + f.deleteOnExit(); + + OutputStream fileOutStream = new FileOutputStream(f); + DataOutput outStream = new DataOutputStream(fileOutStream); + + HCatRecord[] recs = getHCatRecords(); + for (int i = 0; i < recs.length; i++) { + recs[i].write(outStream); + } + fileOutStream.flush(); + fileOutStream.close(); + + InputStream fInStream = new FileInputStream(f); + DataInput inpStream = new DataInputStream(fInStream); + + for (int i = 0; i < recs.length; i++) { + HCatRecord rec = new DefaultHCatRecord(); + rec.readFields(inpStream); + Assert.assertTrue(HCatDataCheckUtil.recordsEqual(recs[i], rec)); + } + + Assert.assertEquals(fInStream.available(), 0); + fInStream.close(); + + } + + public void testCompareTo() { + HCatRecord[] recs = getHCatRecords(); + Assert.assertTrue(HCatDataCheckUtil.compareRecords(recs[0], recs[1]) == 0); + Assert.assertTrue(HCatDataCheckUtil.compareRecords(recs[4], recs[5]) == 0); + } + + public void testEqualsObject() { + + HCatRecord[] recs = getHCatRecords(); + Assert.assertTrue(HCatDataCheckUtil.recordsEqual(recs[0], recs[1])); + Assert.assertTrue(HCatDataCheckUtil.recordsEqual(recs[4], recs[5])); + } + + /** + * Test get and set calls with type + * @throws HCatException + */ + public void testGetSetByType1() throws HCatException { + HCatRecord inpRec = getHCatRecords()[0]; + HCatRecord newRec = new DefaultHCatRecord(inpRec.size()); + HCatSchema hsch = + HCatSchemaUtils.getHCatSchema( + "a:tinyint,b:smallint,c:int,d:bigint,e:float,f:double,g:boolean,h:string,i:binary,j:string"); + + + newRec.setByte("a", hsch, inpRec.getByte("a", hsch)); + newRec.setShort("b", hsch, inpRec.getShort("b", hsch)); + newRec.setInteger("c", hsch, inpRec.getInteger("c", hsch)); + newRec.setLong("d", hsch, inpRec.getLong("d", hsch)); + newRec.setFloat("e", hsch, inpRec.getFloat("e", hsch)); + newRec.setDouble("f", hsch, inpRec.getDouble("f", hsch)); + newRec.setBoolean("g", hsch, inpRec.getBoolean("g", hsch)); + newRec.setString("h", hsch, inpRec.getString("h", hsch)); + newRec.setByteArray("i", hsch, inpRec.getByteArray("i", hsch)); + newRec.setString("j", hsch, inpRec.getString("j", hsch)); + + Assert.assertTrue(HCatDataCheckUtil.recordsEqual(newRec, inpRec)); + + + } + + /** + * Test get and set calls with type + * @throws HCatException + */ + public void testGetSetByType2() throws HCatException { + HCatRecord inpRec = getGetSet2InpRec(); + + HCatRecord newRec = new DefaultHCatRecord(inpRec.size()); + HCatSchema hsch = + HCatSchemaUtils.getHCatSchema("a:binary,b:map,c:array,d:struct"); + + + newRec.setByteArray("a", hsch, inpRec.getByteArray("a", hsch)); + newRec.setMap("b", hsch, inpRec.getMap("b", hsch)); + newRec.setList("c", hsch, inpRec.getList("c", hsch)); + newRec.setStruct("d", hsch, inpRec.getStruct("d", hsch)); + + Assert.assertTrue(HCatDataCheckUtil.recordsEqual(newRec, inpRec)); + } + + + private HCatRecord getGetSet2InpRec() { + List rlist = new ArrayList(); + + rlist.add(new byte[]{1, 2, 3}); + + Map mapcol = new HashMap(3); + mapcol.put(new Short("2"), "hcat is cool"); + mapcol.put(new Short("3"), "is it?"); + mapcol.put(new Short("4"), "or is it not?"); + rlist.add(mapcol); + + List listcol = new ArrayList(); + listcol.add(314); + listcol.add(007); + rlist.add(listcol);//list + rlist.add(listcol);//struct + return new DefaultHCatRecord(rlist); + } + + private HCatRecord[] getHCatRecords() { + + List rec_1 = new ArrayList(8); + rec_1.add(new Byte("123")); + rec_1.add(new Short("456")); + rec_1.add(new Integer(789)); + rec_1.add(new Long(1000L)); + rec_1.add(new Float(5.3F)); + rec_1.add(new Double(5.3D)); + rec_1.add(new Boolean(true)); + rec_1.add(new String("hcat and hadoop")); + rec_1.add(null); + rec_1.add("null"); + + HCatRecord tup_1 = new DefaultHCatRecord(rec_1); + + List rec_2 = new ArrayList(8); + rec_2.add(new Byte("123")); + rec_2.add(new Short("456")); + rec_2.add(new Integer(789)); + rec_2.add(new Long(1000L)); + rec_2.add(new Float(5.3F)); + rec_2.add(new Double(5.3D)); + rec_2.add(new Boolean(true)); + rec_2.add(new String("hcat and hadoop")); + rec_2.add(null); + rec_2.add("null"); + HCatRecord tup_2 = new DefaultHCatRecord(rec_2); + + List rec_3 = new ArrayList(10); + rec_3.add(new Byte("123")); + rec_3.add(new Short("456")); + rec_3.add(new Integer(789)); + rec_3.add(new Long(1000L)); + rec_3.add(new Double(5.3D)); + rec_3.add(new String("hcat and hadoop")); + rec_3.add(null); + List innerList = new ArrayList(); + innerList.add(314); + innerList.add(007); + rec_3.add(innerList); + Map map = new HashMap(3); + map.put(new Short("2"), "hcat is cool"); + map.put(new Short("3"), "is it?"); + map.put(new Short("4"), "or is it not?"); + rec_3.add(map); + + HCatRecord tup_3 = new DefaultHCatRecord(rec_3); + + List rec_4 = new ArrayList(8); + rec_4.add(new Byte("123")); + rec_4.add(new Short("456")); + rec_4.add(new Integer(789)); + rec_4.add(new Long(1000L)); + rec_4.add(new Double(5.3D)); + rec_4.add(new String("hcat and hadoop")); + rec_4.add(null); + rec_4.add("null"); + + Map map2 = new HashMap(3); + map2.put(new Short("2"), "hcat is cool"); + map2.put(new Short("3"), "is it?"); + map2.put(new Short("4"), "or is it not?"); + rec_4.add(map2); + List innerList2 = new ArrayList(); + innerList2.add(314); + innerList2.add(007); + rec_4.add(innerList2); + HCatRecord tup_4 = new DefaultHCatRecord(rec_4); + + + List rec_5 = new ArrayList(3); + rec_5.add(getByteArray()); + rec_5.add(getStruct()); + rec_5.add(getList()); + HCatRecord tup_5 = new DefaultHCatRecord(rec_5); + + + List rec_6 = new ArrayList(3); + rec_6.add(getByteArray()); + rec_6.add(getStruct()); + rec_6.add(getList()); + HCatRecord tup_6 = new DefaultHCatRecord(rec_6); + + + return new HCatRecord[]{tup_1, tup_2, tup_3, tup_4, tup_5, tup_6}; + + } + + private Object getList() { + return getStruct(); + } + + private Object getByteArray() { + return new byte[]{1, 2, 3, 4}; + } + + private List getStruct() { + List struct = new ArrayList(); + struct.add(new Integer(1)); + struct.add(new String("x")); + return struct; + } +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/data/TestHCatRecordSerDe.java hcatalog/core/src/test/java/org/apache/hcatalog/data/TestHCatRecordSerDe.java new file mode 100644 index 0000000..8f3e801 --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/data/TestHCatRecordSerDe.java @@ -0,0 +1,172 @@ +/** + * 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.hcatalog.data; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Properties; + +import junit.framework.Assert; +import junit.framework.TestCase; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.serde.serdeConstants; +import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe; +import org.apache.hadoop.io.Writable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.data.TestHCatRecordSerDe} instead + */ +public class TestHCatRecordSerDe extends TestCase { + + private static final Logger LOG = LoggerFactory.getLogger(TestHCatRecordSerDe.class); + + public Map getData() { + Map data = new HashMap(); + + List rlist = new ArrayList(11); + rlist.add(new Byte("123")); + rlist.add(new Short("456")); + rlist.add(new Integer(789)); + rlist.add(new Long(1000L)); + rlist.add(new Double(5.3D)); + rlist.add(new Float(2.39F)); + rlist.add(new String("hcat and hadoop")); + rlist.add(null); + + List innerStruct = new ArrayList(2); + innerStruct.add(new String("abc")); + innerStruct.add(new String("def")); + rlist.add(innerStruct); + + List innerList = new ArrayList(); + innerList.add(314); + innerList.add(007); + rlist.add(innerList); + + Map map = new HashMap(3); + map.put(new Short("2"), "hcat is cool"); + map.put(new Short("3"), "is it?"); + map.put(new Short("4"), "or is it not?"); + rlist.add(map); + + rlist.add(new Boolean(true)); + + List c1 = new ArrayList(); + List c1_1 = new ArrayList(); + c1_1.add(new Integer(12)); + List i2 = new ArrayList(); + List ii1 = new ArrayList(); + ii1.add(new Integer(13)); + ii1.add(new Integer(14)); + i2.add(ii1); + Map> ii2 = new HashMap>(); + List iii1 = new ArrayList(); + iii1.add(new Integer(15)); + ii2.put("phew", iii1); + i2.add(ii2); + c1_1.add(i2); + c1.add(c1_1); + rlist.add(c1); + List am = new ArrayList(); + Map am_1 = new HashMap(); + am_1.put("noo", "haha"); + am.add(am_1); + rlist.add(am); + List aa = new ArrayList(); + List aa_1 = new ArrayList(); + aa_1.add("bloo"); + aa_1.add("bwahaha"); + aa.add(aa_1); + rlist.add(aa); + + String typeString = + "tinyint,smallint,int,bigint,double,float,string,string," + + "struct,array,map,boolean," + + "array,ii2:map>>>>," + + "array>,array>"; + Properties props = new Properties(); + + props.put(serdeConstants.LIST_COLUMNS, "ti,si,i,bi,d,f,s,n,r,l,m,b,c1,am,aa"); + props.put(serdeConstants.LIST_COLUMN_TYPES, typeString); +// props.put(Constants.SERIALIZATION_NULL_FORMAT, "\\N"); +// props.put(Constants.SERIALIZATION_FORMAT, "1"); + + data.put(props, new DefaultHCatRecord(rlist)); + return data; + } + + public void testRW() throws Exception { + + Configuration conf = new Configuration(); + + for (Entry e : getData().entrySet()) { + Properties tblProps = e.getKey(); + HCatRecord r = e.getValue(); + + HCatRecordSerDe hrsd = new HCatRecordSerDe(); + hrsd.initialize(conf, tblProps); + + LOG.info("ORIG: {}", r); + + Writable s = hrsd.serialize(r, hrsd.getObjectInspector()); + LOG.info("ONE: {}", s); + + HCatRecord r2 = (HCatRecord) hrsd.deserialize(s); + Assert.assertTrue(HCatDataCheckUtil.recordsEqual(r, r2)); + + // If it went through correctly, then s is also a HCatRecord, + // and also equal to the above, and a deepcopy, and this holds + // through for multiple levels more of serialization as well. + + Writable s2 = hrsd.serialize(s, hrsd.getObjectInspector()); + LOG.info("TWO: {}", s2); + Assert.assertTrue(HCatDataCheckUtil.recordsEqual(r, (HCatRecord) s)); + Assert.assertTrue(HCatDataCheckUtil.recordsEqual(r, (HCatRecord) s2)); + + // serialize using another serde, and read out that object repr. + LazySimpleSerDe testSD = new LazySimpleSerDe(); + testSD.initialize(conf, tblProps); + + Writable s3 = testSD.serialize(s, hrsd.getObjectInspector()); + LOG.info("THREE: {}", s3); + Object o3 = testSD.deserialize(s3); + Assert.assertFalse(r.getClass().equals(o3.getClass())); + + // then serialize again using hrsd, and compare results + HCatRecord s4 = (HCatRecord) hrsd.serialize(o3, testSD.getObjectInspector()); + LOG.info("FOUR: {}", s4); + + // Test LazyHCatRecord init and read + LazyHCatRecord s5 = new LazyHCatRecord(o3, testSD.getObjectInspector()); + LOG.info("FIVE: {}", s5); + + LazyHCatRecord s6 = new LazyHCatRecord(s4, hrsd.getObjectInspector()); + LOG.info("SIX: {}", s6); + + } + + } + +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/data/TestJsonSerDe.java hcatalog/core/src/test/java/org/apache/hcatalog/data/TestJsonSerDe.java new file mode 100644 index 0000000..f41ca36 --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/data/TestJsonSerDe.java @@ -0,0 +1,217 @@ +/** + * 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.hcatalog.data; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import junit.framework.TestCase; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.serde.serdeConstants; +import org.apache.hadoop.io.Writable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.data.TestJsonSerDe} instead + */ +public class TestJsonSerDe extends TestCase { + + private static final Logger LOG = LoggerFactory.getLogger(TestJsonSerDe.class); + + public List> getData() { + List> data = new ArrayList>(); + + List rlist = new ArrayList(13); + rlist.add(new Byte("123")); + rlist.add(new Short("456")); + rlist.add(new Integer(789)); + rlist.add(new Long(1000L)); + rlist.add(new Double(5.3D)); + rlist.add(new Float(2.39F)); + rlist.add(new String("hcat and hadoop")); + rlist.add(null); + + List innerStruct = new ArrayList(2); + innerStruct.add(new String("abc")); + innerStruct.add(new String("def")); + rlist.add(innerStruct); + + List innerList = new ArrayList(); + innerList.add(314); + innerList.add(007); + rlist.add(innerList); + + Map map = new HashMap(3); + map.put(new Short("2"), "hcat is cool"); + map.put(new Short("3"), "is it?"); + map.put(new Short("4"), "or is it not?"); + rlist.add(map); + + rlist.add(new Boolean(true)); + + List c1 = new ArrayList(); + List c1_1 = new ArrayList(); + c1_1.add(new Integer(12)); + List i2 = new ArrayList(); + List ii1 = new ArrayList(); + ii1.add(new Integer(13)); + ii1.add(new Integer(14)); + i2.add(ii1); + Map> ii2 = new HashMap>(); + List iii1 = new ArrayList(); + iii1.add(new Integer(15)); + ii2.put("phew", iii1); + i2.add(ii2); + c1_1.add(i2); + c1.add(c1_1); + rlist.add(c1); + + List nlist = new ArrayList(13); + nlist.add(null); // tinyint + nlist.add(null); // smallint + nlist.add(null); // int + nlist.add(null); // bigint + nlist.add(null); // double + nlist.add(null); // float + nlist.add(null); // string + nlist.add(null); // string + nlist.add(null); // struct + nlist.add(null); // array + nlist.add(null); // map + nlist.add(null); // bool + nlist.add(null); // complex + + String typeString = + "tinyint,smallint,int,bigint,double,float,string,string," + + "struct,array,map,boolean," + + "array,ii2:map>>>>"; + Properties props = new Properties(); + + props.put(serdeConstants.LIST_COLUMNS, "ti,si,i,bi,d,f,s,n,r,l,m,b,c1"); + props.put(serdeConstants.LIST_COLUMN_TYPES, typeString); +// props.put(Constants.SERIALIZATION_NULL_FORMAT, "\\N"); +// props.put(Constants.SERIALIZATION_FORMAT, "1"); + + data.add(new Pair(props, new DefaultHCatRecord(rlist))); + data.add(new Pair(props, new DefaultHCatRecord(nlist))); + return data; + } + + public void testRW() throws Exception { + + Configuration conf = new Configuration(); + + for (Pair e : getData()) { + Properties tblProps = e.first; + HCatRecord r = e.second; + + HCatRecordSerDe hrsd = new HCatRecordSerDe(); + hrsd.initialize(conf, tblProps); + + JsonSerDe jsde = new JsonSerDe(); + jsde.initialize(conf, tblProps); + + LOG.info("ORIG:{}", r); + + Writable s = hrsd.serialize(r, hrsd.getObjectInspector()); + LOG.info("ONE:{}", s); + + Object o1 = hrsd.deserialize(s); + assertTrue(HCatDataCheckUtil.recordsEqual(r, (HCatRecord) o1)); + + Writable s2 = jsde.serialize(o1, hrsd.getObjectInspector()); + LOG.info("TWO:{}", s2); + Object o2 = jsde.deserialize(s2); + LOG.info("deserialized TWO : {} ", o2); + + assertTrue(HCatDataCheckUtil.recordsEqual(r, (HCatRecord) o2)); + } + + } + + public void testRobustRead() throws Exception { + /** + * This test has been added to account for HCATALOG-436 + * We write out columns with "internal column names" such + * as "_col0", but try to read with retular column names. + */ + + Configuration conf = new Configuration(); + + for (Pair e : getData()) { + Properties tblProps = e.first; + HCatRecord r = e.second; + + Properties internalTblProps = new Properties(); + for (Map.Entry pe : tblProps.entrySet()) { + if (!pe.getKey().equals(serdeConstants.LIST_COLUMNS)) { + internalTblProps.put(pe.getKey(), pe.getValue()); + } else { + internalTblProps.put(pe.getKey(), getInternalNames((String) pe.getValue())); + } + } + + LOG.info("orig tbl props:{}", tblProps); + LOG.info("modif tbl props:{}", internalTblProps); + + JsonSerDe wjsd = new JsonSerDe(); + wjsd.initialize(conf, internalTblProps); + + JsonSerDe rjsd = new JsonSerDe(); + rjsd.initialize(conf, tblProps); + + LOG.info("ORIG:{}", r); + + Writable s = wjsd.serialize(r, wjsd.getObjectInspector()); + LOG.info("ONE:{}", s); + + Object o1 = wjsd.deserialize(s); + LOG.info("deserialized ONE : {} ", o1); + + Object o2 = rjsd.deserialize(s); + LOG.info("deserialized TWO : {} ", o2); + assertTrue(HCatDataCheckUtil.recordsEqual(r, (HCatRecord) o2)); + } + + } + + String getInternalNames(String columnNames) { + if (columnNames == null) { + return null; + } + if (columnNames.isEmpty()) { + return ""; + } + + StringBuffer sb = new StringBuffer(); + int numStrings = columnNames.split(",").length; + sb.append("_col0"); + for (int i = 1; i < numStrings; i++) { + sb.append(","); + sb.append(HiveConf.getColumnInternalName(i)); + } + return sb.toString(); + } +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/data/TestLazyHCatRecord.java hcatalog/core/src/test/java/org/apache/hcatalog/data/TestLazyHCatRecord.java new file mode 100644 index 0000000..ed0147c --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/data/TestLazyHCatRecord.java @@ -0,0 +1,196 @@ +/** + * 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.hcatalog.data; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; +import org.apache.hcatalog.data.schema.HCatSchema; +import org.apache.hcatalog.data.schema.HCatSchemaUtils; +import org.junit.Assert; +import org.junit.Test; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.data.TestLazyHCatRecord} instead + */ +public class TestLazyHCatRecord { + + private final int INT_CONST = 789; + private final long LONG_CONST = 5000000000L; + private final double DOUBLE_CONST = 3.141592654; + private final String STRING_CONST = "hello world"; + + @Test + public void testGet() throws Exception { + HCatRecord r = new LazyHCatRecord(getHCatRecord(), getObjectInspector()); + Assert.assertEquals(INT_CONST, ((Integer) r.get(0)).intValue()); + Assert.assertEquals(LONG_CONST, ((Long) r.get(1)).longValue()); + Assert.assertEquals(DOUBLE_CONST, ((Double) r.get(2)).doubleValue(), 0); + Assert.assertEquals(STRING_CONST, (String) r.get(3)); + } + + @Test + public void testGetWithName() throws Exception { + TypeInfo ti = getTypeInfo(); + HCatRecord r = new LazyHCatRecord(getHCatRecord(), getObjectInspector(ti)); + HCatSchema schema = HCatSchemaUtils.getHCatSchema(ti) + .get(0).getStructSubSchema(); + Assert.assertEquals(INT_CONST, ((Integer) r.get("an_int", schema)).intValue()); + Assert.assertEquals(LONG_CONST, ((Long) r.get("a_long", schema)).longValue()); + Assert.assertEquals(DOUBLE_CONST, ((Double) r.get("a_double", schema)).doubleValue(), 0); + Assert.assertEquals(STRING_CONST, (String) r.get("a_string", schema)); + } + + @Test + public void testGetAll() throws Exception { + HCatRecord r = new LazyHCatRecord(getHCatRecord(), getObjectInspector()); + List list = r.getAll(); + Assert.assertEquals(INT_CONST, ((Integer) list.get(0)).intValue()); + Assert.assertEquals(LONG_CONST, ((Long) list.get(1)).longValue()); + Assert.assertEquals(DOUBLE_CONST, ((Double) list.get(2)).doubleValue(), 0); + Assert.assertEquals(STRING_CONST, (String) list.get(3)); + } + + @Test + public void testSet() throws Exception { + HCatRecord r = new LazyHCatRecord(getHCatRecord(), getObjectInspector()); + boolean sawException = false; + try { + r.set(3, "Mary had a little lamb"); + } catch (UnsupportedOperationException uoe) { + sawException = true; + } + Assert.assertTrue(sawException); + } + + @Test + public void testSize() throws Exception { + HCatRecord r = new LazyHCatRecord(getHCatRecord(), getObjectInspector()); + Assert.assertEquals(4, r.size()); + } + + @Test + public void testReadFields() throws Exception { + HCatRecord r = new LazyHCatRecord(getHCatRecord(), getObjectInspector()); + boolean sawException = false; + try { + r.readFields(null); + } catch (UnsupportedOperationException uoe) { + sawException = true; + } + Assert.assertTrue(sawException); + } + + @Test + public void testWrite() throws Exception { + HCatRecord r = new LazyHCatRecord(getHCatRecord(), getObjectInspector()); + boolean sawException = false; + try { + r.write(null); + } catch (UnsupportedOperationException uoe) { + sawException = true; + } + Assert.assertTrue(sawException); + } + + @Test + public void testSetWithName() throws Exception { + HCatRecord r = new LazyHCatRecord(getHCatRecord(), getObjectInspector()); + boolean sawException = false; + try { + r.set("fred", null, "bob"); + } catch (UnsupportedOperationException uoe) { + sawException = true; + } + Assert.assertTrue(sawException); + } + + @Test + public void testRemove() throws Exception { + HCatRecord r = new LazyHCatRecord(getHCatRecord(), getObjectInspector()); + boolean sawException = false; + try { + r.remove(0); + } catch (UnsupportedOperationException uoe) { + sawException = true; + } + Assert.assertTrue(sawException); + } + + @Test + public void testCopy() throws Exception { + HCatRecord r = new LazyHCatRecord(getHCatRecord(), getObjectInspector()); + boolean sawException = false; + try { + r.copy(null); + } catch (UnsupportedOperationException uoe) { + sawException = true; + } + Assert.assertTrue(sawException); + } + + @Test + public void testGetWritable() throws Exception { + HCatRecord r = new LazyHCatRecord(getHCatRecord(), getObjectInspector()).getWritable(); + Assert.assertEquals(INT_CONST, ((Integer) r.get(0)).intValue()); + Assert.assertEquals(LONG_CONST, ((Long) r.get(1)).longValue()); + Assert.assertEquals(DOUBLE_CONST, ((Double) r.get(2)).doubleValue(), 0); + Assert.assertEquals(STRING_CONST, (String) r.get(3)); + Assert.assertEquals("org.apache.hcatalog.data.DefaultHCatRecord", r.getClass().getName()); + } + + private HCatRecord getHCatRecord() throws Exception { + List rec_1 = new ArrayList(4); + rec_1.add( new Integer(INT_CONST)); + rec_1.add( new Long(LONG_CONST)); + rec_1.add( new Double(DOUBLE_CONST)); + rec_1.add( new String(STRING_CONST)); + + return new DefaultHCatRecord(rec_1); + } + + private TypeInfo getTypeInfo() throws Exception { + List names = new ArrayList(4); + names.add("an_int"); + names.add("a_long"); + names.add("a_double"); + names.add("a_string"); + + List tis = new ArrayList(4); + tis.add(TypeInfoFactory.getPrimitiveTypeInfo("int")); + tis.add(TypeInfoFactory.getPrimitiveTypeInfo("bigint")); + tis.add(TypeInfoFactory.getPrimitiveTypeInfo("double")); + tis.add(TypeInfoFactory.getPrimitiveTypeInfo("string")); + + return TypeInfoFactory.getStructTypeInfo(names, tis); + } + + private ObjectInspector getObjectInspector(TypeInfo ti) throws Exception { + return HCatRecordObjectInspectorFactory.getHCatRecordObjectInspector((StructTypeInfo)ti); + } + + private ObjectInspector getObjectInspector() throws Exception { + return HCatRecordObjectInspectorFactory.getHCatRecordObjectInspector( + (StructTypeInfo)getTypeInfo()); + } +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/data/TestReaderWriter.java hcatalog/core/src/test/java/org/apache/hcatalog/data/TestReaderWriter.java new file mode 100644 index 0000000..2c28bdd --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/data/TestReaderWriter.java @@ -0,0 +1,186 @@ +/** + * 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.hcatalog.data; + +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.ql.CommandNeedRetryException; +import org.apache.hadoop.mapreduce.InputSplit; +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.data.transfer.DataTransferFactory; +import org.apache.hcatalog.data.transfer.HCatReader; +import org.apache.hcatalog.data.transfer.HCatWriter; +import org.apache.hcatalog.data.transfer.ReadEntity; +import org.apache.hcatalog.data.transfer.ReaderContext; +import org.apache.hcatalog.data.transfer.WriteEntity; +import org.apache.hcatalog.data.transfer.WriterContext; +import org.apache.hcatalog.mapreduce.HCatBaseTest; +import org.junit.Assert; +import org.junit.Test; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.data.TestReaderWriter} instead + */ +public class TestReaderWriter extends HCatBaseTest { + + @Test + public void test() throws MetaException, CommandNeedRetryException, + IOException, ClassNotFoundException { + + driver.run("drop table mytbl"); + driver.run("create table mytbl (a string, b int)"); + Iterator> itr = hiveConf.iterator(); + Map map = new HashMap(); + while (itr.hasNext()) { + Entry kv = itr.next(); + map.put(kv.getKey(), kv.getValue()); + } + + WriterContext cntxt = runsInMaster(map); + + File writeCntxtFile = File.createTempFile("hcat-write", "temp"); + writeCntxtFile.deleteOnExit(); + + // Serialize context. + ObjectOutputStream oos = new ObjectOutputStream(new FileOutputStream(writeCntxtFile)); + oos.writeObject(cntxt); + oos.flush(); + oos.close(); + + // Now, deserialize it. + ObjectInputStream ois = new ObjectInputStream(new FileInputStream(writeCntxtFile)); + cntxt = (WriterContext) ois.readObject(); + ois.close(); + + runsInSlave(cntxt); + commit(map, true, cntxt); + + ReaderContext readCntxt = runsInMaster(map, false); + + File readCntxtFile = File.createTempFile("hcat-read", "temp"); + readCntxtFile.deleteOnExit(); + oos = new ObjectOutputStream(new FileOutputStream(readCntxtFile)); + oos.writeObject(readCntxt); + oos.flush(); + oos.close(); + + ois = new ObjectInputStream(new FileInputStream(readCntxtFile)); + readCntxt = (ReaderContext) ois.readObject(); + ois.close(); + + for (InputSplit split : readCntxt.getSplits()) { + runsInSlave(split, readCntxt.getConf()); + } + } + + private WriterContext runsInMaster(Map config) throws HCatException { + + WriteEntity.Builder builder = new WriteEntity.Builder(); + WriteEntity entity = builder.withTable("mytbl").build(); + HCatWriter writer = DataTransferFactory.getHCatWriter(entity, config); + WriterContext info = writer.prepareWrite(); + return info; + } + + private ReaderContext runsInMaster(Map config, boolean bogus) + throws HCatException { + ReadEntity entity = new ReadEntity.Builder().withTable("mytbl").build(); + HCatReader reader = DataTransferFactory.getHCatReader(entity, config); + ReaderContext cntxt = reader.prepareRead(); + return cntxt; + } + + private void runsInSlave(InputSplit split, Configuration config) throws HCatException { + + HCatReader reader = DataTransferFactory.getHCatReader(split, config); + Iterator itr = reader.read(); + int i = 1; + while (itr.hasNext()) { + HCatRecord read = itr.next(); + HCatRecord written = getRecord(i++); + // Argh, HCatRecord doesnt implement equals() + Assert.assertTrue("Read: " + read.get(0) + "Written: " + written.get(0), + written.get(0).equals(read.get(0))); + Assert.assertTrue("Read: " + read.get(1) + "Written: " + written.get(1), + written.get(1).equals(read.get(1))); + Assert.assertEquals(2, read.size()); + } + //Assert.assertFalse(itr.hasNext()); + } + + private void runsInSlave(WriterContext context) throws HCatException { + + HCatWriter writer = DataTransferFactory.getHCatWriter(context); + writer.write(new HCatRecordItr()); + } + + private void commit(Map config, boolean status, + WriterContext context) throws IOException { + + WriteEntity.Builder builder = new WriteEntity.Builder(); + WriteEntity entity = builder.withTable("mytbl").build(); + HCatWriter writer = DataTransferFactory.getHCatWriter(entity, config); + if (status) { + writer.commit(context); + } else { + writer.abort(context); + } + } + + private static HCatRecord getRecord(int i) { + List list = new ArrayList(2); + list.add("Row #: " + i); + list.add(i); + return new DefaultHCatRecord(list); + } + + private static class HCatRecordItr implements Iterator { + + int i = 0; + + @Override + public boolean hasNext() { + return i++ < 100 ? true : false; + } + + @Override + public HCatRecord next() { + return getRecord(i); + } + + @Override + public void remove() { + throw new RuntimeException(); + } + } +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/data/schema/TestHCatSchema.java hcatalog/core/src/test/java/org/apache/hcatalog/data/schema/TestHCatSchema.java new file mode 100644 index 0000000..83868fc --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/data/schema/TestHCatSchema.java @@ -0,0 +1,81 @@ +/** + * 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.hcatalog.data.schema; + +import junit.framework.TestCase; +import org.apache.hcatalog.common.HCatException; + +import java.util.ArrayList; +import java.util.List; +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.data.schema.TestHCatSchema} instead + */ +public class TestHCatSchema extends TestCase { + public void testCannotAddFieldMoreThanOnce() throws HCatException { + List fieldSchemaList = new ArrayList(); + fieldSchemaList.add(new HCatFieldSchema("name", HCatFieldSchema.Type.STRING, "What's your handle?")); + fieldSchemaList.add(new HCatFieldSchema("age", HCatFieldSchema.Type.INT, "So very old")); + + HCatSchema schema = new HCatSchema(fieldSchemaList); + + assertTrue(schema.getFieldNames().contains("age")); + assertEquals(2, schema.getFields().size()); + + try { + schema.append(new HCatFieldSchema("age", HCatFieldSchema.Type.INT, "So very old")); + fail("Was able to append field schema with same name"); + } catch (HCatException he) { + assertTrue(he.getMessage().contains("Attempt to append HCatFieldSchema with already existing name: age.")); + } + + assertTrue(schema.getFieldNames().contains("age")); + assertEquals(2, schema.getFields().size()); + + // Should also not be able to add fields of different types with same name + try { + schema.append(new HCatFieldSchema("age", HCatFieldSchema.Type.STRING, "Maybe spelled out?")); + fail("Was able to append field schema with same name"); + } catch (HCatException he) { + assertTrue(he.getMessage().contains("Attempt to append HCatFieldSchema with already existing name: age.")); + } + + assertTrue(schema.getFieldNames().contains("age")); + assertEquals(2, schema.getFields().size()); + } + + public void testCannotInstantiateSchemaWithRepeatedFieldNames() throws HCatException { + List fieldSchemaList = new ArrayList(); + + fieldSchemaList.add(new HCatFieldSchema("memberID", HCatFieldSchema.Type.INT, "as a number")); + fieldSchemaList.add(new HCatFieldSchema("location", HCatFieldSchema.Type.STRING, "there's Waldo")); + + // No duplicate names. This should be ok + HCatSchema schema = new HCatSchema(fieldSchemaList); + + fieldSchemaList.add(new HCatFieldSchema("memberID", HCatFieldSchema.Type.STRING, "as a String")); + + // Now a duplicated field name. Should fail + try { + HCatSchema schema2 = new HCatSchema(fieldSchemaList); + fail("Able to add duplicate field name"); + } catch (IllegalArgumentException iae) { + assertTrue(iae.getMessage().contains("Field named memberID already exists")); + } + } +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/data/schema/TestHCatSchemaUtils.java hcatalog/core/src/test/java/org/apache/hcatalog/data/schema/TestHCatSchemaUtils.java new file mode 100644 index 0000000..a5f14d6 --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/data/schema/TestHCatSchemaUtils.java @@ -0,0 +1,84 @@ +/** + * 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.hcatalog.data.schema; + +import java.io.PrintStream; + +import junit.framework.TestCase; + +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils; +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.data.schema.HCatFieldSchema.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.data.schema.TestHCatSchema} instead + */ +public class TestHCatSchemaUtils extends TestCase { + + private static final Logger LOG = LoggerFactory.getLogger(TestHCatSchemaUtils.class); + + public void testSimpleOperation() throws Exception { + String typeString = "struct," + + "currently_registered_courses:array," + + "current_grades:map," + + "phnos:array>,blah:array>"; + + TypeInfo ti = TypeInfoUtils.getTypeInfoFromTypeString(typeString); + + HCatSchema hsch = HCatSchemaUtils.getHCatSchemaFromTypeString(typeString); + LOG.info("Type name : {}", ti.getTypeName()); + LOG.info("HCatSchema : {}", hsch); + assertEquals(hsch.size(), 1); + assertEquals(ti.getTypeName(), hsch.get(0).getTypeString()); + assertEquals(hsch.get(0).getTypeString(), typeString); + } + + @SuppressWarnings("unused") + private void pretty_print(PrintStream pout, HCatSchema hsch) throws HCatException { + pretty_print(pout, hsch, ""); + } + + + private void pretty_print(PrintStream pout, HCatSchema hsch, String prefix) throws HCatException { + int i = 0; + for (HCatFieldSchema field : hsch.getFields()) { + pretty_print(pout, field, prefix + "." + (field.getName() == null ? i : field.getName())); + i++; + } + } + + private void pretty_print(PrintStream pout, HCatFieldSchema hfsch, String prefix) throws HCatException { + + Category tcat = hfsch.getCategory(); + if (Category.STRUCT == tcat) { + pretty_print(pout, hfsch.getStructSubSchema(), prefix); + } else if (Category.ARRAY == tcat) { + pretty_print(pout, hfsch.getArrayElementSchema(), prefix); + } else if (Category.MAP == tcat) { + pout.println(prefix + ".mapkey:\t" + hfsch.getMapKeyType().toString()); + pretty_print(pout, hfsch.getMapValueSchema(), prefix + ".mapvalue:"); + } else { + pout.println(prefix + "\t" + hfsch.getType().toString()); + } + } + +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/fileformats/TestOrcDynamicPartitioned.java hcatalog/core/src/test/java/org/apache/hcatalog/fileformats/TestOrcDynamicPartitioned.java new file mode 100644 index 0000000..dd55557 --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/fileformats/TestOrcDynamicPartitioned.java @@ -0,0 +1,55 @@ +/** + * 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.hcatalog.fileformats; + +import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat; +import org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat; +import org.apache.hadoop.hive.ql.io.orc.OrcSerde; +import org.apache.hcatalog.mapreduce.TestHCatDynamicPartitioned; +import org.junit.BeforeClass; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.fileformats.TestOrcDynamicPartitioned} instead + */ +public class TestOrcDynamicPartitioned extends TestHCatDynamicPartitioned { + + @BeforeClass + public static void generateInputData() throws Exception { + tableName = "testOrcDynamicPartitionedTable"; + generateWriteRecords(NUM_RECORDS, NUM_PARTITIONS, 0); + generateDataColumns(); + } + + @Override + protected String inputFormat() { + return OrcInputFormat.class.getName(); + } + + @Override + protected String outputFormat() { + return OrcOutputFormat.class.getName(); + } + + @Override + protected String serdeClass() { + return OrcSerde.class.getName(); + } + +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/mapred/TestHiveHCatInputFormat.java.broken hcatalog/core/src/test/java/org/apache/hcatalog/mapred/TestHiveHCatInputFormat.java.broken new file mode 100644 index 0000000..082d723 --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/mapred/TestHiveHCatInputFormat.java.broken @@ -0,0 +1,193 @@ +/** + * 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.hcatalog.mapred; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Properties; + +import junit.framework.TestCase; + +import org.apache.hadoop.hive.ql.CommandNeedRetryException; +import org.apache.hadoop.hive.ql.Driver; +import org.apache.hadoop.hive.ql.io.RCFileInputFormat; +import org.apache.hadoop.hive.ql.io.RCFileOutputFormat; +import org.apache.hcatalog.MiniCluster; +import org.apache.hcatalog.data.HCatDataCheckUtil; +import org.apache.hcatalog.mapred.HCatMapredInputFormat; +import org.apache.hcatalog.mapreduce.HCatInputFormat; +import org.apache.hcatalog.storagehandler.HCatStorageHandlerImpl; +import org.apache.pig.ExecType; +import org.apache.pig.PigServer; +import org.apache.pig.impl.util.UDFContext; + +public class TestHiveHCatInputFormat extends TestCase { + private static MiniCluster cluster = MiniCluster.buildCluster(); + private static Driver driver; + + String PTNED_TABLE = "junit_testhiveinputintegration_ptni"; + String UNPTNED_TABLE = "junit_testhiveinputintegration_noptn"; + String basicFile = "/tmp/"+PTNED_TABLE+".file"; + + public void testFromHive() throws Exception { + if (driver == null){ + driver = HCatDataCheckUtil.instantiateDriver(cluster); + } + + Properties props = new Properties(); + props.setProperty("fs.default.name", cluster.getProperties().getProperty("fs.default.name")); + String basicFileFullName = cluster.getProperties().getProperty("fs.default.name") + basicFile; + + cleanup(); + + // create source data file + HCatDataCheckUtil.generateDataFile(cluster,basicFile); + + String createPtnedTable = "(j int, s string) partitioned by (i int) " + +"stored by '"+HCatStorageHandlerImpl.class.getName()+"' tblproperties" + + "('hcat.isd'='org.apache.hcatalog.rcfile.RCFileInputDriver'," + + "'hcat.osd'='org.apache.hcatalog.rcfile.RCFileOutputDriver') "; + + HCatDataCheckUtil.createTable(driver,PTNED_TABLE,createPtnedTable); + + String createUnptnedTable = "(i int, j int, s string) " + +"stored by '"+HCatStorageHandlerImpl.class.getName()+"' tblproperties" + + "('hcat.isd'='org.apache.hcatalog.rcfile.RCFileInputDriver'," + + "'hcat.osd'='org.apache.hcatalog.rcfile.RCFileOutputDriver') "; + + HCatDataCheckUtil.createTable(driver,UNPTNED_TABLE,createUnptnedTable); + + + driver.run("describe extended "+UNPTNED_TABLE); + ArrayList des_values = new ArrayList(); + driver.getResults(des_values); + for (String s : des_values){ + System.err.println("du:"+s); + } + + driver.run("describe extended "+PTNED_TABLE); + ArrayList des2_values = new ArrayList(); + driver.getResults(des2_values); + for (String s : des2_values){ + System.err.println("dp:"+s); + } + + // use pig to read from source file and put into this table + + PigServer server = new PigServer(ExecType.LOCAL, props); + UDFContext.getUDFContext().setClientSystemProps(); + server.setBatchOn(); + server.registerQuery("A = load '"+basicFileFullName+"' as (i:int, j:int, s:chararray);"); + server.registerQuery("store A into '"+UNPTNED_TABLE+"' using org.apache.hcatalog.pig.HCatStorer();"); + server.executeBatch(); + + server.setBatchOn(); + server.registerQuery("A = load '"+basicFileFullName+"' as (i:int, j:int, s:chararray);"); + server.registerQuery("store A into '"+PTNED_TABLE+"' using org.apache.hcatalog.pig.HCatStorer();"); + server.executeBatch(); + + // partitioned by i + // select * from tbl; + // select j,s,i from tbl; + // select * from tbl where i = 3; + // select j,s,i from tbl where i = 3; + // select * from tbl where j = 3; + // select j,s,i from tbl where j = 3; + + ArrayList p_select_star_nofilter = HCatDataCheckUtil.formattedRun(driver, + "p_select_star_nofilter","select * from "+PTNED_TABLE); + ArrayList p_select_named_nofilter = HCatDataCheckUtil.formattedRun(driver, + "p_select_named_nofilter","select j,s,i from "+PTNED_TABLE); + + assertDataIdentical(p_select_star_nofilter,p_select_named_nofilter,50); + + ArrayList p_select_star_ptnfilter = HCatDataCheckUtil.formattedRun(driver, + "p_select_star_ptnfilter","select * from "+PTNED_TABLE+" where i = 3"); + ArrayList p_select_named_ptnfilter = HCatDataCheckUtil.formattedRun(driver, + "p_select_named_ptnfilter","select j,s,i from "+PTNED_TABLE+" where i = 3"); + + assertDataIdentical(p_select_star_ptnfilter,p_select_named_ptnfilter,10); + + ArrayList select_star_nonptnfilter = HCatDataCheckUtil.formattedRun(driver, + "select_star_nonptnfilter","select * from "+PTNED_TABLE+" where j = 28"); + ArrayList select_named_nonptnfilter = HCatDataCheckUtil.formattedRun(driver, + "select_named_nonptnfilter","select j,s,i from "+PTNED_TABLE+" where j = 28"); + + assertDataIdentical(select_star_nonptnfilter,select_named_nonptnfilter,1); + + // non-partitioned + // select * from tbl; + // select i,j,s from tbl; + // select * from tbl where i = 3; + // select i,j,s from tbl where i = 3; + + // select j,s,i from tbl; + // select j,s,i from tbl where i = 3; + + ArrayList select_star_nofilter = HCatDataCheckUtil.formattedRun(driver, + "select_star_nofilter","select * from "+UNPTNED_TABLE); //i,j,s select * order is diff for unptn + ArrayList select_ijs_nofilter = HCatDataCheckUtil.formattedRun(driver, + "select_ijs_nofilter","select i,j,s from "+UNPTNED_TABLE); + + assertDataIdentical(select_star_nofilter,select_ijs_nofilter,50); + + ArrayList select_star_ptnfilter = HCatDataCheckUtil.formattedRun(driver, + "select_star_ptnfilter","select * from "+UNPTNED_TABLE+" where i = 3"); //i,j,s + ArrayList select_ijs_ptnfilter = HCatDataCheckUtil.formattedRun(driver, + "select_ijs_ptnfilter","select i,j,s from "+UNPTNED_TABLE+" where i = 3"); + + assertDataIdentical(select_star_ptnfilter,select_ijs_ptnfilter,10); + + ArrayList select_jsi_nofilter = HCatDataCheckUtil.formattedRun(driver, + "select_jsi_nofilter","select j,s,i from "+UNPTNED_TABLE); + assertDataIdentical(p_select_named_nofilter,select_jsi_nofilter,50,true); + + ArrayList select_jsi_ptnfilter = HCatDataCheckUtil.formattedRun(driver, + "select_jsi_ptnfilter","select j,s,i from "+UNPTNED_TABLE+" where i = 3"); + assertDataIdentical(p_select_named_ptnfilter,select_jsi_ptnfilter,10,true); + + } + + private void assertDataIdentical(ArrayList result1, + ArrayList result2, int numRecords) { + assertDataIdentical(result1,result2,numRecords,false); + } + + private void assertDataIdentical(ArrayList result1, + ArrayList result2, int numRecords,boolean doSort) { + assertEquals(numRecords, result1.size()); + assertEquals(numRecords, result2.size()); + Collections.sort(result1); + Collections.sort(result2); + for (int i = 0; i < numRecords; i++){ + assertEquals(result1.get(i),result2.get(i)); + } + } + + + private void cleanup() throws IOException, CommandNeedRetryException { + MiniCluster.deleteFile(cluster, basicFile); + HCatDataCheckUtil.dropTable(driver,PTNED_TABLE); + HCatDataCheckUtil.dropTable(driver,UNPTNED_TABLE); + } + +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/HCatBaseTest.java hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/HCatBaseTest.java new file mode 100644 index 0000000..f582580 --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/HCatBaseTest.java @@ -0,0 +1,87 @@ +/** + * 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.hcatalog.mapreduce; + +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.hive.cli.CliSessionState; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.ql.Driver; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.pig.PigServer; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; + +/** + * Simplify writing HCatalog tests that require a HiveMetaStore. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.HCatBaseTest} instead + */ +public class HCatBaseTest { + protected static final Logger LOG = LoggerFactory.getLogger(HCatBaseTest.class); + protected static final String TEST_DATA_DIR = System.getProperty("user.dir") + + "/build/test/data/" + HCatBaseTest.class.getCanonicalName(); + protected static final String TEST_WAREHOUSE_DIR = TEST_DATA_DIR + "/warehouse"; + + protected HiveConf hiveConf = null; + protected Driver driver = null; + protected HiveMetaStoreClient client = null; + + @BeforeClass + public static void setUpTestDataDir() throws Exception { + LOG.info("Using warehouse directory " + TEST_WAREHOUSE_DIR); + File f = new File(TEST_WAREHOUSE_DIR); + if (f.exists()) { + FileUtil.fullyDelete(f); + } + Assert.assertTrue(new File(TEST_WAREHOUSE_DIR).mkdirs()); + } + + @Before + public void setUp() throws Exception { + if (driver == null) { + setUpHiveConf(); + driver = new Driver(hiveConf); + client = new HiveMetaStoreClient(hiveConf); + SessionState.start(new CliSessionState(hiveConf)); + } + } + + /** + * Create a new HiveConf and set properties necessary for unit tests. + */ + protected void setUpHiveConf() { + hiveConf = new HiveConf(this.getClass()); + hiveConf.setVar(HiveConf.ConfVars.PREEXECHOOKS, ""); + hiveConf.setVar(HiveConf.ConfVars.POSTEXECHOOKS, ""); + hiveConf.setBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, false); + hiveConf.setVar(HiveConf.ConfVars.METASTOREWAREHOUSE, TEST_WAREHOUSE_DIR); + } + + protected void logAndRegister(PigServer server, String query) throws IOException { + LOG.info("Registering pig query: " + query); + server.registerQuery(query); + } +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/HCatMapReduceTest.java hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/HCatMapReduceTest.java new file mode 100644 index 0000000..82bd1e2 --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/HCatMapReduceTest.java @@ -0,0 +1,372 @@ +/** + * 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.hcatalog.mapreduce; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import junit.framework.Assert; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocalFileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.MetaStoreUtils; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.SerDeInfo; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.ql.io.RCFileInputFormat; +import org.apache.hadoop.hive.ql.io.RCFileOutputFormat; +import org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe; +import org.apache.hadoop.hive.serde.serdeConstants; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.WritableComparable; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.JobStatus; +import org.apache.hadoop.mapreduce.Mapper; +import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; +import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat; +import org.apache.hcatalog.common.HCatConstants; +import org.apache.hcatalog.common.HCatUtil; +import org.apache.hcatalog.data.DefaultHCatRecord; +import org.apache.hcatalog.data.HCatRecord; +import org.apache.hcatalog.data.schema.HCatFieldSchema; +import org.apache.hcatalog.data.schema.HCatSchema; +import org.junit.After; +import org.junit.Before; +import org.junit.BeforeClass; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.junit.Assert.assertTrue; + +/** + * Test for HCatOutputFormat. Writes a partition using HCatOutputFormat and reads + * it back using HCatInputFormat, checks the column values and counts. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.HCatMapReduceTest} instead + */ +public abstract class HCatMapReduceTest extends HCatBaseTest { + + private static final Logger LOG = LoggerFactory.getLogger(HCatMapReduceTest.class); + protected static String dbName = MetaStoreUtils.DEFAULT_DATABASE_NAME; + protected static String tableName = "testHCatMapReduceTable"; + + private static List writeRecords = new ArrayList(); + private static List readRecords = new ArrayList(); + + protected abstract List getPartitionKeys(); + + protected abstract List getTableColumns(); + + private static FileSystem fs; + + protected String inputFormat() { + return RCFileInputFormat.class.getName(); + } + + protected String outputFormat() { + return RCFileOutputFormat.class.getName(); + } + + protected String serdeClass() { + return ColumnarSerDe.class.getName(); + } + + @BeforeClass + public static void setUpOneTime() throws Exception { + fs = new LocalFileSystem(); + fs.initialize(fs.getWorkingDirectory().toUri(), new Configuration()); + + HiveConf hiveConf = new HiveConf(); + hiveConf.setInt(HCatConstants.HCAT_HIVE_CLIENT_EXPIRY_TIME, 0); + // Hack to initialize cache with 0 expiry time causing it to return a new hive client every time + // Otherwise the cache doesn't play well with the second test method with the client gets closed() in the + // tearDown() of the previous test + HCatUtil.getHiveClient(hiveConf); + + MapCreate.writeCount = 0; + MapRead.readCount = 0; + } + + @After + public void deleteTable() throws Exception { + try { + String databaseName = (dbName == null) ? MetaStoreUtils.DEFAULT_DATABASE_NAME : dbName; + + client.dropTable(databaseName, tableName); + } catch (Exception e) { + e.printStackTrace(); + throw e; + } + } + + @Before + public void createTable() throws Exception { + String databaseName = (dbName == null) ? MetaStoreUtils.DEFAULT_DATABASE_NAME : dbName; + + try { + client.dropTable(databaseName, tableName); + } catch (Exception e) { + } //can fail with NoSuchObjectException + + + Table tbl = new Table(); + tbl.setDbName(databaseName); + tbl.setTableName(tableName); + tbl.setTableType("MANAGED_TABLE"); + StorageDescriptor sd = new StorageDescriptor(); + + sd.setCols(getTableColumns()); + tbl.setPartitionKeys(getPartitionKeys()); + + tbl.setSd(sd); + + sd.setBucketCols(new ArrayList(2)); + sd.setSerdeInfo(new SerDeInfo()); + sd.getSerdeInfo().setName(tbl.getTableName()); + sd.getSerdeInfo().setParameters(new HashMap()); + sd.getSerdeInfo().getParameters().put(serdeConstants.SERIALIZATION_FORMAT, "1"); + sd.getSerdeInfo().setSerializationLib(serdeClass()); + sd.setInputFormat(inputFormat()); + sd.setOutputFormat(outputFormat()); + + Map tableParams = new HashMap(); + tbl.setParameters(tableParams); + + client.createTable(tbl); + } + + //Create test input file with specified number of rows + private void createInputFile(Path path, int rowCount) throws IOException { + + if (fs.exists(path)) { + fs.delete(path, true); + } + + FSDataOutputStream os = fs.create(path); + + for (int i = 0; i < rowCount; i++) { + os.writeChars(i + "\n"); + } + + os.close(); + } + + public static class MapCreate extends + Mapper { + + static int writeCount = 0; //test will be in local mode + + @Override + public void map(LongWritable key, Text value, Context context + ) throws IOException, InterruptedException { + { + try { + HCatRecord rec = writeRecords.get(writeCount); + context.write(null, rec); + writeCount++; + + } catch (Exception e) { + + e.printStackTrace(System.err); //print since otherwise exception is lost + throw new IOException(e); + } + } + } + } + + public static class MapRead extends + Mapper { + + static int readCount = 0; //test will be in local mode + + @Override + public void map(WritableComparable key, HCatRecord value, Context context + ) throws IOException, InterruptedException { + { + try { + readRecords.add(value); + readCount++; + } catch (Exception e) { + e.printStackTrace(); //print since otherwise exception is lost + throw new IOException(e); + } + } + } + } + + Job runMRCreate(Map partitionValues, + List partitionColumns, List records, + int writeCount, boolean assertWrite) throws Exception { + return runMRCreate(partitionValues, partitionColumns, records, writeCount, assertWrite, true); + } + + /** + * Run a local map reduce job to load data from in memory records to an HCatalog Table + * @param partitionValues + * @param partitionColumns + * @param records data to be written to HCatalog table + * @param writeCount + * @param assertWrite + * @param asSingleMapTask + * @return + * @throws Exception + */ + Job runMRCreate(Map partitionValues, + List partitionColumns, List records, + int writeCount, boolean assertWrite, boolean asSingleMapTask) throws Exception { + + writeRecords = records; + MapCreate.writeCount = 0; + + Configuration conf = new Configuration(); + Job job = new Job(conf, "hcat mapreduce write test"); + job.setJarByClass(this.getClass()); + job.setMapperClass(HCatMapReduceTest.MapCreate.class); + + // input/output settings + job.setInputFormatClass(TextInputFormat.class); + + if (asSingleMapTask) { + // One input path would mean only one map task + Path path = new Path(fs.getWorkingDirectory(), "mapred/testHCatMapReduceInput"); + createInputFile(path, writeCount); + TextInputFormat.setInputPaths(job, path); + } else { + // Create two input paths so that two map tasks get triggered. There could be other ways + // to trigger two map tasks. + Path path = new Path(fs.getWorkingDirectory(), "mapred/testHCatMapReduceInput"); + createInputFile(path, writeCount / 2); + + Path path2 = new Path(fs.getWorkingDirectory(), "mapred/testHCatMapReduceInput2"); + createInputFile(path2, (writeCount - writeCount / 2)); + + TextInputFormat.setInputPaths(job, path, path2); + } + + job.setOutputFormatClass(HCatOutputFormat.class); + + OutputJobInfo outputJobInfo = OutputJobInfo.create(dbName, tableName, partitionValues); + HCatOutputFormat.setOutput(job, outputJobInfo); + + job.setMapOutputKeyClass(BytesWritable.class); + job.setMapOutputValueClass(DefaultHCatRecord.class); + + job.setNumReduceTasks(0); + + HCatOutputFormat.setSchema(job, new HCatSchema(partitionColumns)); + + boolean success = job.waitForCompletion(true); + + // Ensure counters are set when data has actually been read. + if (partitionValues != null) { + assertTrue(job.getCounters().getGroup("FileSystemCounters") + .findCounter("FILE_BYTES_READ").getValue() > 0); + } + + if (!HCatUtil.isHadoop23()) { + // Local mode outputcommitter hook is not invoked in Hadoop 1.x + if (success) { + new FileOutputCommitterContainer(job, null).commitJob(job); + } else { + new FileOutputCommitterContainer(job, null).abortJob(job, JobStatus.State.FAILED); + } + } + if (assertWrite) { + // we assert only if we expected to assert with this call. + Assert.assertEquals(writeCount, MapCreate.writeCount); + } + + return job; + } + + List runMRRead(int readCount) throws Exception { + return runMRRead(readCount, null); + } + + /** + * Run a local map reduce job to read records from HCatalog table and verify if the count is as expected + * @param readCount + * @param filter + * @return + * @throws Exception + */ + List runMRRead(int readCount, String filter) throws Exception { + + MapRead.readCount = 0; + readRecords.clear(); + + Configuration conf = new Configuration(); + Job job = new Job(conf, "hcat mapreduce read test"); + job.setJarByClass(this.getClass()); + job.setMapperClass(HCatMapReduceTest.MapRead.class); + + // input/output settings + job.setInputFormatClass(HCatInputFormat.class); + job.setOutputFormatClass(TextOutputFormat.class); + + HCatInputFormat.setInput(job, dbName, tableName).setFilter(filter); + + job.setMapOutputKeyClass(BytesWritable.class); + job.setMapOutputValueClass(Text.class); + + job.setNumReduceTasks(0); + + Path path = new Path(fs.getWorkingDirectory(), "mapred/testHCatMapReduceOutput"); + if (fs.exists(path)) { + fs.delete(path, true); + } + + TextOutputFormat.setOutputPath(job, path); + + job.waitForCompletion(true); + Assert.assertEquals(readCount, MapRead.readCount); + + return readRecords; + } + + + protected HCatSchema getTableSchema() throws Exception { + + Configuration conf = new Configuration(); + Job job = new Job(conf, "hcat mapreduce read schema test"); + job.setJarByClass(this.getClass()); + + // input/output settings + job.setInputFormatClass(HCatInputFormat.class); + job.setOutputFormatClass(TextOutputFormat.class); + + HCatInputFormat.setInput(job, dbName, tableName); + + return HCatInputFormat.getTableSchema(job); + } + +} + + + diff --git hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatDynamicPartitioned.java hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatDynamicPartitioned.java new file mode 100644 index 0000000..9dbdb5e --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatDynamicPartitioned.java @@ -0,0 +1,211 @@ +/** + * 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.hcatalog.mapreduce; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import junit.framework.Assert; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.serde.serdeConstants; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hcatalog.common.HCatUtil; +import org.apache.hcatalog.common.ErrorType; +import org.apache.hcatalog.common.HCatConstants; +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.data.DefaultHCatRecord; +import org.apache.hcatalog.data.HCatRecord; +import org.apache.hcatalog.data.schema.HCatFieldSchema; +import org.apache.hcatalog.data.schema.HCatSchemaUtils; +import org.junit.BeforeClass; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static junit.framework.Assert.assertEquals; +import static junit.framework.Assert.assertTrue; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.TestHCatExternalDynamicPartitioned} instead + */ +public class TestHCatDynamicPartitioned extends HCatMapReduceTest { + + private static List writeRecords; + private static List dataColumns; + private static final Logger LOG = LoggerFactory.getLogger(TestHCatDynamicPartitioned.class); + protected static final int NUM_RECORDS = 20; + protected static final int NUM_PARTITIONS = 5; + + @BeforeClass + public static void generateInputData() throws Exception { + tableName = "testHCatDynamicPartitionedTable"; + generateWriteRecords(NUM_RECORDS, NUM_PARTITIONS, 0); + generateDataColumns(); + } + + protected static void generateDataColumns() throws HCatException { + dataColumns = new ArrayList(); + dataColumns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("c1", serdeConstants.INT_TYPE_NAME, ""))); + dataColumns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("c2", serdeConstants.STRING_TYPE_NAME, ""))); + dataColumns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("p1", serdeConstants.STRING_TYPE_NAME, ""))); + } + + protected static void generateWriteRecords(int max, int mod, int offset) { + writeRecords = new ArrayList(); + + for (int i = 0; i < max; i++) { + List objList = new ArrayList(); + + objList.add(i); + objList.add("strvalue" + i); + objList.add(String.valueOf((i % mod) + offset)); + writeRecords.add(new DefaultHCatRecord(objList)); + } + } + + @Override + protected List getPartitionKeys() { + List fields = new ArrayList(); + fields.add(new FieldSchema("p1", serdeConstants.STRING_TYPE_NAME, "")); + return fields; + } + + @Override + protected List getTableColumns() { + List fields = new ArrayList(); + fields.add(new FieldSchema("c1", serdeConstants.INT_TYPE_NAME, "")); + fields.add(new FieldSchema("c2", serdeConstants.STRING_TYPE_NAME, "")); + return fields; + } + + /** + * Run the dynamic partitioning test but with single map task + * @throws Exception + */ + @Test + public void testHCatDynamicPartitionedTable() throws Exception { + runHCatDynamicPartitionedTable(true); + } + + /** + * Run the dynamic partitioning test but with multiple map task. See HCATALOG-490 + * @throws Exception + */ + @Test + public void testHCatDynamicPartitionedTableMultipleTask() throws Exception { + runHCatDynamicPartitionedTable(false); + } + + protected void runHCatDynamicPartitionedTable(boolean asSingleMapTask) throws Exception { + generateWriteRecords(NUM_RECORDS, NUM_PARTITIONS, 0); + runMRCreate(null, dataColumns, writeRecords, NUM_RECORDS, true, asSingleMapTask); + + runMRRead(NUM_RECORDS); + + //Read with partition filter + runMRRead(4, "p1 = \"0\""); + runMRRead(8, "p1 = \"1\" or p1 = \"3\""); + runMRRead(4, "p1 = \"4\""); + + // read from hive to test + + String query = "select * from " + tableName; + int retCode = driver.run(query).getResponseCode(); + + if (retCode != 0) { + throw new Exception("Error " + retCode + " running query " + query); + } + + ArrayList res = new ArrayList(); + driver.getResults(res); + assertEquals(NUM_RECORDS, res.size()); + + + //Test for duplicate publish + IOException exc = null; + try { + generateWriteRecords(NUM_RECORDS, NUM_PARTITIONS, 0); + Job job = runMRCreate(null, dataColumns, writeRecords, NUM_RECORDS, false); + + if (HCatUtil.isHadoop23()) { + Assert.assertTrue(job.isSuccessful()==false); + } + } catch (IOException e) { + exc = e; + } + + if (!HCatUtil.isHadoop23()) { + assertTrue(exc != null); + assertTrue(exc instanceof HCatException); + assertTrue("Got exception of type [" + ((HCatException) exc).getErrorType().toString() + + "] Expected ERROR_PUBLISHING_PARTITION or ERROR_MOVE_FAILED", + (ErrorType.ERROR_PUBLISHING_PARTITION == ((HCatException) exc).getErrorType()) + || (ErrorType.ERROR_MOVE_FAILED == ((HCatException) exc).getErrorType()) + ); + } + + query = "show partitions " + tableName; + retCode = driver.run(query).getResponseCode(); + if (retCode != 0) { + throw new Exception("Error " + retCode + " running query " + query); + } + res = new ArrayList(); + driver.getResults(res); + assertEquals(NUM_PARTITIONS, res.size()); + + query = "select * from " + tableName; + retCode = driver.run(query).getResponseCode(); + if (retCode != 0) { + throw new Exception("Error " + retCode + " running query " + query); + } + res = new ArrayList(); + driver.getResults(res); + assertEquals(NUM_RECORDS, res.size()); + } + + //TODO 1.0 miniCluster is slow this test times out, make it work +// renaming test to make test framework skip it + public void _testHCatDynamicPartitionMaxPartitions() throws Exception { + HiveConf hc = new HiveConf(this.getClass()); + + int maxParts = hiveConf.getIntVar(HiveConf.ConfVars.DYNAMICPARTITIONMAXPARTS); + LOG.info("Max partitions allowed = {}", maxParts); + + IOException exc = null; + try { + generateWriteRecords(maxParts + 5, maxParts + 2, 10); + runMRCreate(null, dataColumns, writeRecords, maxParts + 5, false); + } catch (IOException e) { + exc = e; + } + + if (HCatConstants.HCAT_IS_DYNAMIC_MAX_PTN_CHECK_ENABLED) { + assertTrue(exc != null); + assertTrue(exc instanceof HCatException); + assertEquals(ErrorType.ERROR_TOO_MANY_DYNAMIC_PTNS, ((HCatException) exc).getErrorType()); + } else { + assertTrue(exc == null); + runMRRead(maxParts + 5); + } + } +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatEximInputFormat.java.broken hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatEximInputFormat.java.broken new file mode 100644 index 0000000..8b3e089 --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatEximInputFormat.java.broken @@ -0,0 +1,429 @@ +/** + * 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.hcatalog.mapreduce; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +import junit.framework.TestCase; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocalFileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.serde.Constants; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; +import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat; +import org.apache.hcatalog.data.DefaultHCatRecord; +import org.apache.hcatalog.data.HCatRecord; +import org.apache.hcatalog.data.schema.HCatFieldSchema; +import org.apache.hcatalog.data.schema.HCatSchema; +import org.apache.hcatalog.data.schema.HCatSchemaUtils; +import org.apache.hcatalog.mapreduce.TestHCatEximInputFormat.TestImport.EmpDetails; + +/** + * + * TestHCatEximInputFormat. tests primarily HCatEximInputFormat but + * also HCatEximOutputFormat. + * + */ +public class TestHCatEximInputFormat extends TestCase { + + public static class TestExport extends + org.apache.hadoop.mapreduce.Mapper { + + private HCatSchema recordSchema; + + @Override + protected void setup(Context context) throws IOException, + InterruptedException { + super.setup(context); + recordSchema = HCatEximOutputFormat.getTableSchema(context); + } + + @Override + public void map(LongWritable key, Text value, Context context) + throws IOException, InterruptedException { + String[] cols = value.toString().split(","); + HCatRecord record = new DefaultHCatRecord(recordSchema.size()); + record.setInteger("emp_id", recordSchema, Integer.parseInt(cols[0])); + record.setString("emp_name", recordSchema, cols[1]); + record.setString("emp_dob", recordSchema, cols[2]); + record.setString("emp_sex", recordSchema, cols[3]); + context.write(key, record); + } + } + + public static class TestImport extends + org.apache.hadoop.mapreduce.Mapper< + org.apache.hadoop.io.LongWritable, HCatRecord, + org.apache.hadoop.io.Text, + org.apache.hadoop.io.Text> { + + private HCatSchema recordSchema; + + public static class EmpDetails { + public String emp_name; + public String emp_dob; + public String emp_sex; + public String emp_country; + public String emp_state; + } + + public static Map empRecords = new TreeMap(); + + @Override + protected void setup(Context context) throws IOException, + InterruptedException { + super.setup(context); + try { + recordSchema = HCatBaseInputFormat.getOutputSchema(context); + } catch (Exception e) { + throw new IOException("Error getting outputschema from job configuration", e); + } + System.out.println("RecordSchema : " + recordSchema.toString()); + } + + @Override + public void map(LongWritable key, HCatRecord value, Context context) + throws IOException, InterruptedException { + EmpDetails empDetails = new EmpDetails(); + Integer emp_id = value.getInteger("emp_id", recordSchema); + String emp_name = value.getString("emp_name", recordSchema); + empDetails.emp_name = emp_name; + if (recordSchema.getPosition("emp_dob") != null) { + empDetails.emp_dob = value.getString("emp_dob", recordSchema); + } + if (recordSchema.getPosition("emp_sex") != null) { + empDetails.emp_sex = value.getString("emp_sex", recordSchema); + } + if (recordSchema.getPosition("emp_country") != null) { + empDetails.emp_country = value.getString("emp_country", recordSchema); + } + if (recordSchema.getPosition("emp_state") != null) { + empDetails.emp_state = value.getString("emp_state", recordSchema); + } + empRecords.put(emp_id, empDetails); + } + } + + private static final String dbName = "hcatEximOutputFormatTestDB"; + private static final String tblName = "hcatEximOutputFormatTestTable"; + Configuration conf; + Job job; + List columns; + HCatSchema schema; + FileSystem fs; + Path inputLocation; + Path outputLocation; + private HCatSchema partSchema; + + + @Override + protected void setUp() throws Exception { + System.out.println("Setup started"); + super.setUp(); + conf = new Configuration(); + job = new Job(conf, "test eximinputformat"); + columns = new ArrayList(); + columns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("emp_id", + Constants.INT_TYPE_NAME, ""))); + columns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("emp_name", + Constants.STRING_TYPE_NAME, ""))); + columns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("emp_dob", + Constants.STRING_TYPE_NAME, ""))); + columns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("emp_sex", + Constants.STRING_TYPE_NAME, ""))); + schema = new HCatSchema(columns); + + fs = new LocalFileSystem(); + fs.initialize(fs.getWorkingDirectory().toUri(), new Configuration()); + inputLocation = new Path(fs.getWorkingDirectory(), "tmp/exports"); + outputLocation = new Path(fs.getWorkingDirectory(), "tmp/data"); + + job.setJarByClass(this.getClass()); + job.setNumReduceTasks(0); + System.out.println("Setup done"); + } + + private void setupMRExport(String[] records) throws IOException { + if (fs.exists(outputLocation)) { + fs.delete(outputLocation, true); + } + FSDataOutputStream ds = fs.create(outputLocation, true); + for (String record : records) { + ds.writeBytes(record); + } + ds.close(); + job.setInputFormatClass(TextInputFormat.class); + job.setOutputFormatClass(HCatEximOutputFormat.class); + TextInputFormat.setInputPaths(job, outputLocation); + job.setMapperClass(TestExport.class); + } + + private void setupMRImport() throws IOException { + if (fs.exists(outputLocation)) { + fs.delete(outputLocation, true); + } + job.setInputFormatClass(HCatEximInputFormat.class); + job.setOutputFormatClass(TextOutputFormat.class); + TextOutputFormat.setOutputPath(job, outputLocation); + job.setMapperClass(TestImport.class); + TestImport.empRecords.clear(); + } + + + @Override + protected void tearDown() throws Exception { + System.out.println("Teardown started"); + super.tearDown(); + // fs.delete(inputLocation, true); + // fs.delete(outputLocation, true); + System.out.println("Teardown done"); + } + + + private void runNonPartExport() throws IOException, InterruptedException, ClassNotFoundException { + if (fs.exists(inputLocation)) { + fs.delete(inputLocation, true); + } + setupMRExport(new String[] { + "237,Krishna,01/01/1990,M,IN,TN\n", + "238,Kalpana,01/01/2000,F,IN,KA\n", + "239,Satya,01/01/2001,M,US,TN\n", + "240,Kavya,01/01/2002,F,US,KA\n" + + }); + HCatEximOutputFormat.setOutput( + job, + dbName, + tblName, + inputLocation.toString(), + null, + null, + schema); + + job.waitForCompletion(true); + HCatEximOutputCommitter committer = new HCatEximOutputCommitter(job,null); + committer.cleanupJob(job); + } + + private void runPartExport(String record, String country, String state) throws IOException, InterruptedException, ClassNotFoundException { + setupMRExport(new String[] {record}); + List partValues = new ArrayList(2); + partValues.add(country); + partValues.add(state); + HCatEximOutputFormat.setOutput( + job, + dbName, + tblName, + inputLocation.toString(), + partSchema , + partValues , + schema); + + job.waitForCompletion(true); + HCatEximOutputCommitter committer = new HCatEximOutputCommitter(job,null); + committer.cleanupJob(job); + } + + public void testNonPart() throws Exception { + try { + runNonPartExport(); + setUp(); + setupMRImport(); + HCatEximInputFormat.setInput(job, "tmp/exports", null); + job.waitForCompletion(true); + + assertEquals(4, TestImport.empRecords.size()); + assertEmpDetail(TestImport.empRecords.get(237), "Krishna", "01/01/1990", "M", null, null); + assertEmpDetail(TestImport.empRecords.get(238), "Kalpana", "01/01/2000", "F", null, null); + assertEmpDetail(TestImport.empRecords.get(239), "Satya", "01/01/2001", "M", null, null); + assertEmpDetail(TestImport.empRecords.get(240), "Kavya", "01/01/2002", "F", null, null); + } catch (Exception e) { + System.out.println("Test failed with " + e.getMessage()); + e.printStackTrace(); + throw e; + } + } + + public void testNonPartProjection() throws Exception { + try { + + runNonPartExport(); + setUp(); + setupMRImport(); + HCatEximInputFormat.setInput(job, "tmp/exports", null); + + List readColumns = new ArrayList(); + readColumns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("emp_id", + Constants.INT_TYPE_NAME, ""))); + readColumns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("emp_name", + Constants.STRING_TYPE_NAME, ""))); + + HCatEximInputFormat.setOutputSchema(job, new HCatSchema(readColumns)); + job.waitForCompletion(true); + + assertEquals(4, TestImport.empRecords.size()); + assertEmpDetail(TestImport.empRecords.get(237), "Krishna", null, null, null, null); + assertEmpDetail(TestImport.empRecords.get(238), "Kalpana", null, null, null, null); + assertEmpDetail(TestImport.empRecords.get(239), "Satya", null, null, null, null); + assertEmpDetail(TestImport.empRecords.get(240), "Kavya", null, null, null, null); + } catch (Exception e) { + System.out.println("Test failed with " + e.getMessage()); + e.printStackTrace(); + throw e; + } + } + + public void testPart() throws Exception { + try { + if (fs.exists(inputLocation)) { + fs.delete(inputLocation, true); + } + + List partKeys = new ArrayList(2); + partKeys.add(new HCatFieldSchema("emp_country", HCatFieldSchema.Type.STRING, "")); + partKeys.add(new HCatFieldSchema("emp_state", HCatFieldSchema.Type.STRING, "")); + partSchema = new HCatSchema(partKeys); + + runPartExport("237,Krishna,01/01/1990,M,IN,TN", "in", "tn"); + setUp(); + runPartExport("238,Kalpana,01/01/2000,F,IN,KA\n", "in", "ka"); + setUp(); + runPartExport("239,Satya,01/01/2001,M,US,TN\n", "us", "tn"); + setUp(); + runPartExport("240,Kavya,01/01/2002,F,US,KA\n", "us", "ka"); + + setUp(); + setupMRImport(); + HCatEximInputFormat.setInput(job, "tmp/exports", null); + job.waitForCompletion(true); + + assertEquals(4, TestImport.empRecords.size()); + assertEmpDetail(TestImport.empRecords.get(237), "Krishna", "01/01/1990", "M", "in", "tn"); + assertEmpDetail(TestImport.empRecords.get(238), "Kalpana", "01/01/2000", "F", "in", "ka"); + assertEmpDetail(TestImport.empRecords.get(239), "Satya", "01/01/2001", "M", "us", "tn"); + assertEmpDetail(TestImport.empRecords.get(240), "Kavya", "01/01/2002", "F", "us", "ka"); + } catch (Exception e) { + System.out.println("Test failed with " + e.getMessage()); + e.printStackTrace(); + throw e; + } + } + + public void testPartWithPartCols() throws Exception { + try { + if (fs.exists(inputLocation)) { + fs.delete(inputLocation, true); + } + + List partKeys = new ArrayList(2); + partKeys.add(new HCatFieldSchema("emp_country", HCatFieldSchema.Type.STRING, "")); + partKeys.add(new HCatFieldSchema("emp_state", HCatFieldSchema.Type.STRING, "")); + partSchema = new HCatSchema(partKeys); + + runPartExport("237,Krishna,01/01/1990,M,IN,TN", "in", "tn"); + setUp(); + runPartExport("238,Kalpana,01/01/2000,F,IN,KA\n", "in", "ka"); + setUp(); + runPartExport("239,Satya,01/01/2001,M,US,TN\n", "us", "tn"); + setUp(); + runPartExport("240,Kavya,01/01/2002,F,US,KA\n", "us", "ka"); + + setUp(); + setupMRImport(); + HCatEximInputFormat.setInput(job, "tmp/exports", null); + + List colsPlusPartKeys = new ArrayList(); + colsPlusPartKeys.addAll(columns); + colsPlusPartKeys.addAll(partKeys); + + HCatBaseInputFormat.setOutputSchema(job, new HCatSchema(colsPlusPartKeys)); + job.waitForCompletion(true); + + assertEquals(4, TestImport.empRecords.size()); + assertEmpDetail(TestImport.empRecords.get(237), "Krishna", "01/01/1990", "M", "in", "tn"); + assertEmpDetail(TestImport.empRecords.get(238), "Kalpana", "01/01/2000", "F", "in", "ka"); + assertEmpDetail(TestImport.empRecords.get(239), "Satya", "01/01/2001", "M", "us", "tn"); + assertEmpDetail(TestImport.empRecords.get(240), "Kavya", "01/01/2002", "F", "us", "ka"); + } catch (Exception e) { + System.out.println("Test failed with " + e.getMessage()); + e.printStackTrace(); + throw e; + } + } + + + public void testPartSelection() throws Exception { + try { + if (fs.exists(inputLocation)) { + fs.delete(inputLocation, true); + } + + List partKeys = new ArrayList(2); + partKeys.add(new HCatFieldSchema("emp_country", HCatFieldSchema.Type.STRING, "")); + partKeys.add(new HCatFieldSchema("emp_state", HCatFieldSchema.Type.STRING, "")); + partSchema = new HCatSchema(partKeys); + + runPartExport("237,Krishna,01/01/1990,M,IN,TN", "in", "tn"); + setUp(); + runPartExport("238,Kalpana,01/01/2000,F,IN,KA\n", "in", "ka"); + setUp(); + runPartExport("239,Satya,01/01/2001,M,US,TN\n", "us", "tn"); + setUp(); + runPartExport("240,Kavya,01/01/2002,F,US,KA\n", "us", "ka"); + + setUp(); + setupMRImport(); + Map filter = new TreeMap(); + filter.put("emp_state", "ka"); + HCatEximInputFormat.setInput(job, "tmp/exports", filter); + job.waitForCompletion(true); + + assertEquals(2, TestImport.empRecords.size()); + assertEmpDetail(TestImport.empRecords.get(238), "Kalpana", "01/01/2000", "F", "in", "ka"); + assertEmpDetail(TestImport.empRecords.get(240), "Kavya", "01/01/2002", "F", "us", "ka"); + } catch (Exception e) { + System.out.println("Test failed with " + e.getMessage()); + e.printStackTrace(); + throw e; + } + } + + + private void assertEmpDetail(EmpDetails empDetails, String name, String dob, String mf, String country, String state) { + assertNotNull(empDetails); + assertEquals(name, empDetails.emp_name); + assertEquals(dob, empDetails.emp_dob); + assertEquals(mf, empDetails.emp_sex); + assertEquals(country, empDetails.emp_country); + assertEquals(state, empDetails.emp_state); + } + +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatEximOutputFormat.java.broken hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatEximOutputFormat.java.broken new file mode 100644 index 0000000..bf4fb48 --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatEximOutputFormat.java.broken @@ -0,0 +1,261 @@ +/** + * 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.hcatalog.mapreduce; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import junit.framework.TestCase; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocalFileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.ql.parse.EximUtil; +import org.apache.hadoop.hive.serde.Constants; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.Mapper; +import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; +import org.apache.hcatalog.common.HCatConstants; +import org.apache.hcatalog.common.HCatUtil; +import org.apache.hcatalog.data.DefaultHCatRecord; +import org.apache.hcatalog.data.HCatRecord; +import org.apache.hcatalog.data.schema.HCatFieldSchema; +import org.apache.hcatalog.data.schema.HCatSchema; +import org.apache.hcatalog.data.schema.HCatSchemaUtils; + +/** + * + * TestHCatEximOutputFormat. Some basic testing here. More testing done via + * TestHCatEximInputFormat + * + */ +public class TestHCatEximOutputFormat extends TestCase { + + public static class TestMap extends + Mapper { + + private HCatSchema recordSchema; + + @Override + protected void setup(Context context) throws IOException, + InterruptedException { + super.setup(context); + recordSchema = HCatEximOutputFormat.getTableSchema(context); + System.out.println("TestMap/setup called"); + } + + @Override + public void map(LongWritable key, Text value, Context context) + throws IOException, InterruptedException { + String[] cols = value.toString().split(","); + HCatRecord record = new DefaultHCatRecord(recordSchema.size()); + System.out.println("TestMap/map called. Cols[0]:" + cols[0]); + System.out.println("TestMap/map called. Cols[1]:" + cols[1]); + System.out.println("TestMap/map called. Cols[2]:" + cols[2]); + System.out.println("TestMap/map called. Cols[3]:" + cols[3]); + record.setInteger("emp_id", recordSchema, Integer.parseInt(cols[0])); + record.setString("emp_name", recordSchema, cols[1]); + record.setString("emp_dob", recordSchema, cols[2]); + record.setString("emp_sex", recordSchema, cols[3]); + context.write(key, record); + } + } + + + private static final String dbName = "hcatEximOutputFormatTestDB"; + private static final String tblName = "hcatEximOutputFormatTestTable"; + Configuration conf; + Job job; + List columns; + HCatSchema schema; + FileSystem fs; + Path outputLocation; + Path dataLocation; + + public void testNonPart() throws Exception { + try { + HCatEximOutputFormat.setOutput( + job, + dbName, + tblName, + outputLocation.toString(), + null, + null, + schema); + + job.waitForCompletion(true); + HCatEximOutputCommitter committer = new HCatEximOutputCommitter(job,null); + committer.cleanupJob(job); + + Path metadataPath = new Path(outputLocation, "_metadata"); + Map.Entry> rv = EximUtil.readMetaData(fs, metadataPath); + Table table = rv.getKey(); + List partitions = rv.getValue(); + + assertEquals(dbName, table.getDbName()); + assertEquals(tblName, table.getTableName()); + assertTrue(EximUtil.schemaCompare(table.getSd().getCols(), + HCatUtil.getFieldSchemaList(columns))); + assertEquals("org.apache.hcatalog.rcfile.RCFileInputDriver", + table.getParameters().get(HCatConstants.HCAT_ISD_CLASS)); + assertEquals("org.apache.hcatalog.rcfile.RCFileOutputDriver", + table.getParameters().get(HCatConstants.HCAT_OSD_CLASS)); + assertEquals("org.apache.hadoop.hive.ql.io.RCFileInputFormat", + table.getSd().getInputFormat()); + assertEquals("org.apache.hadoop.hive.ql.io.RCFileOutputFormat", + table.getSd().getOutputFormat()); + assertEquals("org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe", + table.getSd().getSerdeInfo().getSerializationLib()); + assertEquals(0, table.getPartitionKeys().size()); + + assertEquals(0, partitions.size()); + } catch (Exception e) { + System.out.println("Test failed with " + e.getMessage()); + e.printStackTrace(); + throw e; + } + + } + + public void testPart() throws Exception { + try { + List partKeys = new ArrayList(); + partKeys.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("emp_country", + Constants.STRING_TYPE_NAME, ""))); + partKeys.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("emp_state", + Constants.STRING_TYPE_NAME, ""))); + HCatSchema partitionSchema = new HCatSchema(partKeys); + + List partitionVals = new ArrayList(); + partitionVals.add("IN"); + partitionVals.add("TN"); + + HCatEximOutputFormat.setOutput( + job, + dbName, + tblName, + outputLocation.toString(), + partitionSchema, + partitionVals, + schema); + + job.waitForCompletion(true); + HCatEximOutputCommitter committer = new HCatEximOutputCommitter(job,null); + committer.cleanupJob(job); + Path metadataPath = new Path(outputLocation, "_metadata"); + Map.Entry> rv = EximUtil.readMetaData(fs, metadataPath); + Table table = rv.getKey(); + List partitions = rv.getValue(); + + assertEquals(dbName, table.getDbName()); + assertEquals(tblName, table.getTableName()); + assertTrue(EximUtil.schemaCompare(table.getSd().getCols(), + HCatUtil.getFieldSchemaList(columns))); + assertEquals("org.apache.hcatalog.rcfile.RCFileInputDriver", + table.getParameters().get(HCatConstants.HCAT_ISD_CLASS)); + assertEquals("org.apache.hcatalog.rcfile.RCFileOutputDriver", + table.getParameters().get(HCatConstants.HCAT_OSD_CLASS)); + assertEquals("org.apache.hadoop.hive.ql.io.RCFileInputFormat", + table.getSd().getInputFormat()); + assertEquals("org.apache.hadoop.hive.ql.io.RCFileOutputFormat", + table.getSd().getOutputFormat()); + assertEquals("org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe", + table.getSd().getSerdeInfo().getSerializationLib()); + assertEquals(2, table.getPartitionKeys().size()); + List partSchema = table.getPartitionKeys(); + assertEquals("emp_country", partSchema.get(0).getName()); + assertEquals("emp_state", partSchema.get(1).getName()); + + assertEquals(1, partitions.size()); + Partition partition = partitions.get(0); + assertEquals("IN", partition.getValues().get(0)); + assertEquals("TN", partition.getValues().get(1)); + assertEquals("org.apache.hcatalog.rcfile.RCFileInputDriver", + partition.getParameters().get(HCatConstants.HCAT_ISD_CLASS)); + assertEquals("org.apache.hcatalog.rcfile.RCFileOutputDriver", + partition.getParameters().get(HCatConstants.HCAT_OSD_CLASS)); + } catch (Exception e) { + System.out.println("Test failed with " + e.getMessage()); + e.printStackTrace(); + throw e; + } + } + + @Override + protected void setUp() throws Exception { + System.out.println("Setup started"); + super.setUp(); + conf = new Configuration(); + job = new Job(conf, "test eximoutputformat"); + columns = new ArrayList(); + columns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("emp_id", + Constants.INT_TYPE_NAME, ""))); + columns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("emp_name", + Constants.STRING_TYPE_NAME, ""))); + columns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("emp_dob", + Constants.STRING_TYPE_NAME, ""))); + columns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("emp_sex", + Constants.STRING_TYPE_NAME, ""))); + schema = new HCatSchema(columns); + + fs = new LocalFileSystem(); + fs.initialize(fs.getWorkingDirectory().toUri(), new Configuration()); + outputLocation = new Path(fs.getWorkingDirectory(), "tmp/exports"); + if (fs.exists(outputLocation)) { + fs.delete(outputLocation, true); + } + dataLocation = new Path(fs.getWorkingDirectory(), "tmp/data"); + if (fs.exists(dataLocation)) { + fs.delete(dataLocation, true); + } + FSDataOutputStream ds = fs.create(dataLocation, true); + ds.writeBytes("237,Krishna,01/01/1990,M,IN,TN\n"); + ds.writeBytes("238,Kalpana,01/01/2000,F,IN,KA\n"); + ds.writeBytes("239,Satya,01/01/2001,M,US,TN\n"); + ds.writeBytes("240,Kavya,01/01/2002,F,US,KA\n"); + ds.close(); + + job.setInputFormatClass(TextInputFormat.class); + job.setOutputFormatClass(HCatEximOutputFormat.class); + TextInputFormat.setInputPaths(job, dataLocation); + job.setJarByClass(this.getClass()); + job.setMapperClass(TestMap.class); + job.setNumReduceTasks(0); + System.out.println("Setup done"); + } + + @Override + protected void tearDown() throws Exception { + System.out.println("Teardown started"); + super.tearDown(); + fs.delete(dataLocation, true); + fs.delete(outputLocation, true); + System.out.println("Teardown done"); + } +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatHiveCompatibility.java hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatHiveCompatibility.java new file mode 100644 index 0000000..c3cace6 --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatHiveCompatibility.java @@ -0,0 +1,132 @@ +/** + * 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.hcatalog.mapreduce; + +import java.io.File; +import java.io.FileWriter; +import java.util.Arrays; +import java.util.Iterator; + +import junit.framework.Assert; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hcatalog.common.HCatConstants; +import org.apache.pig.ExecType; +import org.apache.pig.PigServer; +import org.apache.pig.data.Tuple; +import org.junit.BeforeClass; +import org.junit.Test; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.TestHCatHiveCompatibility} instead + */ +public class TestHCatHiveCompatibility extends HCatBaseTest { + private static final String INPUT_FILE_NAME = TEST_DATA_DIR + "/input.data"; + + @BeforeClass + public static void createInputData() throws Exception { + int LOOP_SIZE = 11; + File file = new File(INPUT_FILE_NAME); + file.deleteOnExit(); + FileWriter writer = new FileWriter(file); + for (int i = 0; i < LOOP_SIZE; i++) { + writer.write(i + "\t1\n"); + } + writer.close(); + } + + @Test + public void testUnpartedReadWrite() throws Exception { + + driver.run("drop table if exists junit_unparted_noisd"); + String createTable = "create table junit_unparted_noisd(a int) stored as RCFILE"; + Assert.assertEquals(0, driver.run(createTable).getResponseCode()); + + // assert that the table created has no hcat instrumentation, and that we're still able to read it. + Table table = client.getTable("default", "junit_unparted_noisd"); + Assert.assertTrue(table.getSd().getInputFormat().equals(HCatConstants.HIVE_RCFILE_IF_CLASS)); + + PigServer server = new PigServer(ExecType.LOCAL); + logAndRegister(server, "A = load '" + INPUT_FILE_NAME + "' as (a:int);"); + logAndRegister(server, "store A into 'default.junit_unparted_noisd' using org.apache.hcatalog.pig.HCatStorer();"); + logAndRegister(server, "B = load 'default.junit_unparted_noisd' using org.apache.hcatalog.pig.HCatLoader();"); + Iterator itr = server.openIterator("B"); + + int i = 0; + + while (itr.hasNext()) { + Tuple t = itr.next(); + Assert.assertEquals(1, t.size()); + Assert.assertEquals(t.get(0), i); + i++; + } + + Assert.assertFalse(itr.hasNext()); + Assert.assertEquals(11, i); + + // assert that the table created still has no hcat instrumentation + Table table2 = client.getTable("default", "junit_unparted_noisd"); + Assert.assertTrue(table2.getSd().getInputFormat().equals(HCatConstants.HIVE_RCFILE_IF_CLASS)); + + driver.run("drop table junit_unparted_noisd"); + } + + @Test + public void testPartedRead() throws Exception { + + driver.run("drop table if exists junit_parted_noisd"); + String createTable = "create table junit_parted_noisd(a int) partitioned by (b string) stored as RCFILE"; + Assert.assertEquals(0, driver.run(createTable).getResponseCode()); + + // assert that the table created has no hcat instrumentation, and that we're still able to read it. + Table table = client.getTable("default", "junit_parted_noisd"); + Assert.assertTrue(table.getSd().getInputFormat().equals(HCatConstants.HIVE_RCFILE_IF_CLASS)); + + PigServer server = new PigServer(ExecType.LOCAL); + logAndRegister(server, "A = load '" + INPUT_FILE_NAME + "' as (a:int);"); + logAndRegister(server, "store A into 'default.junit_parted_noisd' using org.apache.hcatalog.pig.HCatStorer('b=42');"); + logAndRegister(server, "B = load 'default.junit_parted_noisd' using org.apache.hcatalog.pig.HCatLoader();"); + Iterator itr = server.openIterator("B"); + + int i = 0; + + while (itr.hasNext()) { + Tuple t = itr.next(); + Assert.assertEquals(2, t.size()); // Contains explicit field "a" and partition "b". + Assert.assertEquals(t.get(0), i); + Assert.assertEquals(t.get(1), "42"); + i++; + } + + Assert.assertFalse(itr.hasNext()); + Assert.assertEquals(11, i); + + // assert that the table created still has no hcat instrumentation + Table table2 = client.getTable("default", "junit_parted_noisd"); + Assert.assertTrue(table2.getSd().getInputFormat().equals(HCatConstants.HIVE_RCFILE_IF_CLASS)); + + // assert that there is one partition present, and it had hcat instrumentation inserted when it was created. + Partition ptn = client.getPartition("default", "junit_parted_noisd", Arrays.asList("42")); + + Assert.assertNotNull(ptn); + Assert.assertTrue(ptn.getSd().getInputFormat().equals(HCatConstants.HIVE_RCFILE_IF_CLASS)); + driver.run("drop table junit_unparted_noisd"); + } +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatHiveThriftCompatibility.java hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatHiveThriftCompatibility.java new file mode 100644 index 0000000..eebc16f --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatHiveThriftCompatibility.java @@ -0,0 +1,119 @@ +/** + * 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.hcatalog.mapreduce; + +import junit.framework.Assert; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.serde2.thrift.test.IntString; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.SequenceFile; +import org.apache.pig.ExecType; +import org.apache.pig.PigServer; +import org.apache.pig.data.DataType; +import org.apache.pig.data.Tuple; +import org.apache.pig.impl.logicalLayer.schema.Schema; +import org.apache.thrift.protocol.TBinaryProtocol; +import org.apache.thrift.transport.TIOStreamTransport; +import org.junit.Before; +import org.junit.Test; + +import java.io.ByteArrayOutputStream; +import java.util.Iterator; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.TestHCatHiveThriftCompatibility} instead + */ +public class TestHCatHiveThriftCompatibility extends HCatBaseTest { + + private boolean setUpComplete = false; + private Path intStringSeq; + + @Before + @Override + public void setUp() throws Exception { + super.setUp(); + if (setUpComplete) { + return; + } + + ByteArrayOutputStream out = new ByteArrayOutputStream(); + TIOStreamTransport transport = new TIOStreamTransport(out); + TBinaryProtocol protocol = new TBinaryProtocol(transport); + + IntString intString = new IntString(1, "one", 1); + intString.write(protocol); + BytesWritable bytesWritable = new BytesWritable(out.toByteArray()); + + intStringSeq = new Path(TEST_DATA_DIR + "/data/intString.seq"); + LOG.info("Creating data file: " + intStringSeq); + + SequenceFile.Writer seqFileWriter = SequenceFile.createWriter( + intStringSeq.getFileSystem(hiveConf), hiveConf, intStringSeq, + NullWritable.class, BytesWritable.class); + seqFileWriter.append(NullWritable.get(), bytesWritable); + seqFileWriter.close(); + + setUpComplete = true; + } + + /** + * Create a table with no explicit schema and ensure its correctly + * discovered from the thrift struct. + */ + @Test + public void testDynamicCols() throws Exception { + Assert.assertEquals(0, driver.run("drop table if exists test_thrift").getResponseCode()); + Assert.assertEquals(0, driver.run( + "create external table test_thrift " + + "partitioned by (year string) " + + "row format serde 'org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer' " + + "with serdeproperties ( " + + " 'serialization.class'='org.apache.hadoop.hive.serde2.thrift.test.IntString', " + + " 'serialization.format'='org.apache.thrift.protocol.TBinaryProtocol') " + + "stored as" + + " inputformat 'org.apache.hadoop.mapred.SequenceFileInputFormat'" + + " outputformat 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat'") + .getResponseCode()); + Assert.assertEquals(0, + driver.run("alter table test_thrift add partition (year = '2012') location '" + + intStringSeq.getParent() + "'").getResponseCode()); + + PigServer pigServer = new PigServer(ExecType.LOCAL); + pigServer.registerQuery("A = load 'test_thrift' using org.apache.hcatalog.pig.HCatLoader();"); + + Schema expectedSchema = new Schema(); + expectedSchema.add(new Schema.FieldSchema("myint", DataType.INTEGER)); + expectedSchema.add(new Schema.FieldSchema("mystring", DataType.CHARARRAY)); + expectedSchema.add(new Schema.FieldSchema("underscore_int", DataType.INTEGER)); + expectedSchema.add(new Schema.FieldSchema("year", DataType.CHARARRAY)); + + Assert.assertEquals(expectedSchema, pigServer.dumpSchema("A")); + + Iterator iterator = pigServer.openIterator("A"); + Tuple t = iterator.next(); + Assert.assertEquals(1, t.get(0)); + Assert.assertEquals("one", t.get(1)); + Assert.assertEquals(1, t.get(2)); + Assert.assertEquals("2012", t.get(3)); + + Assert.assertFalse(iterator.hasNext()); + } +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatInputFormat.java hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatInputFormat.java new file mode 100644 index 0000000..2cd7065 --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatInputFormat.java @@ -0,0 +1,151 @@ +/** + * 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.hcatalog.mapreduce; + +import junit.framework.Assert; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.serde2.thrift.test.IntString; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.Mapper; +import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat; +import org.apache.hcatalog.common.HCatConstants; +import org.apache.hcatalog.data.HCatRecord; +import org.apache.thrift.protocol.TBinaryProtocol; +import org.apache.thrift.transport.TIOStreamTransport; +import org.junit.Before; +import org.junit.Test; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.TestHCatInputFormat} instead + */ +public class TestHCatInputFormat extends HCatBaseTest { + + private boolean setUpComplete = false; + + /** + * Create an input sequence file with 100 records; every 10th record is bad. + * Load this table into Hive. + */ + @Before + @Override + public void setUp() throws Exception { + super.setUp(); + if (setUpComplete) { + return; + } + + Path intStringSeq = new Path(TEST_DATA_DIR + "/data/intString.seq"); + LOG.info("Creating data file: " + intStringSeq); + SequenceFile.Writer seqFileWriter = SequenceFile.createWriter( + intStringSeq.getFileSystem(hiveConf), hiveConf, intStringSeq, + NullWritable.class, BytesWritable.class); + + ByteArrayOutputStream out = new ByteArrayOutputStream(); + TIOStreamTransport transport = new TIOStreamTransport(out); + TBinaryProtocol protocol = new TBinaryProtocol(transport); + + for (int i = 1; i <= 100; i++) { + if (i % 10 == 0) { + seqFileWriter.append(NullWritable.get(), new BytesWritable("bad record".getBytes())); + } else { + out.reset(); + IntString intString = new IntString(i, Integer.toString(i), i); + intString.write(protocol); + BytesWritable bytesWritable = new BytesWritable(out.toByteArray()); + seqFileWriter.append(NullWritable.get(), bytesWritable); + } + } + + seqFileWriter.close(); + + // Now let's load this file into a new Hive table. + Assert.assertEquals(0, driver.run("drop table if exists test_bad_records").getResponseCode()); + Assert.assertEquals(0, driver.run( + "create table test_bad_records " + + "row format serde 'org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer' " + + "with serdeproperties ( " + + " 'serialization.class'='org.apache.hadoop.hive.serde2.thrift.test.IntString', " + + " 'serialization.format'='org.apache.thrift.protocol.TBinaryProtocol') " + + "stored as" + + " inputformat 'org.apache.hadoop.mapred.SequenceFileInputFormat'" + + " outputformat 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat'") + .getResponseCode()); + Assert.assertEquals(0, driver.run("load data local inpath '" + intStringSeq.getParent() + + "' into table test_bad_records").getResponseCode()); + + setUpComplete = true; + } + + @Test + public void testBadRecordHandlingPasses() throws Exception { + Assert.assertTrue(runJob(0.1f)); + } + + @Test + public void testBadRecordHandlingFails() throws Exception { + Assert.assertFalse(runJob(0.01f)); + } + + private boolean runJob(float badRecordThreshold) throws Exception { + Configuration conf = new Configuration(); + + conf.setFloat(HCatConstants.HCAT_INPUT_BAD_RECORD_THRESHOLD_KEY, badRecordThreshold); + + Job job = new Job(conf); + job.setJarByClass(this.getClass()); + job.setMapperClass(MyMapper.class); + + job.setInputFormatClass(HCatInputFormat.class); + job.setOutputFormatClass(TextOutputFormat.class); + + HCatInputFormat.setInput(job, "default", "test_bad_records"); + + job.setMapOutputKeyClass(HCatRecord.class); + job.setMapOutputValueClass(HCatRecord.class); + + job.setNumReduceTasks(0); + + Path path = new Path(TEST_DATA_DIR, "test_bad_record_handling_output"); + if (path.getFileSystem(conf).exists(path)) { + path.getFileSystem(conf).delete(path, true); + } + + TextOutputFormat.setOutputPath(job, path); + + return job.waitForCompletion(true); + } + + public static class MyMapper extends Mapper { + @Override + public void map(NullWritable key, HCatRecord value, Context context) + throws IOException, InterruptedException { + LOG.info("HCatRecord: " + value); + context.write(NullWritable.get(), new Text(value.toString())); + } + } +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatMultiOutputFormat.java hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatMultiOutputFormat.java new file mode 100644 index 0000000..cefcdd6 --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatMultiOutputFormat.java @@ -0,0 +1,433 @@ +/** + * 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.hcatalog.mapreduce; + +import java.io.File; +import java.io.IOException; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Random; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStore; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hadoop.hive.metastore.api.SerDeInfo; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.ql.exec.FetchTask; +import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.metadata.Partition; +import org.apache.hadoop.hive.ql.plan.FetchWork; +import org.apache.hadoop.hive.ql.plan.PartitionDesc; +import org.apache.hadoop.hive.serde.serdeConstants; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.MiniMRCluster; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.Mapper; +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; +import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; +import org.apache.hcatalog.cli.SemanticAnalysis.HCatSemanticAnalyzer; +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.data.DefaultHCatRecord; +import org.apache.hcatalog.data.HCatRecord; +import org.apache.hcatalog.data.schema.HCatFieldSchema; +import org.apache.hcatalog.data.schema.HCatSchema; +import org.apache.hcatalog.data.schema.HCatSchemaUtils; +import org.apache.hcatalog.mapreduce.MultiOutputFormat.JobConfigurer; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.TestHCatMultiOutputFormat} instead + */ +public class TestHCatMultiOutputFormat { + + private static final Logger LOG = LoggerFactory.getLogger(TestHCatMultiOutputFormat.class); + + private static final String DATABASE = "default"; + private static final String[] tableNames = {"test1", "test2", "test3"}; + private static final String[] tablePerms = {"755", "750", "700"}; + private static Path warehousedir = null; + private static HashMap schemaMap = new HashMap(); + private static HiveMetaStoreClient hmsc; + private static MiniMRCluster mrCluster; + private static Configuration mrConf; + private static HiveConf hiveConf; + private static File workDir; + + private static final String msPort = "20199"; + private static Thread t; + + static { + schemaMap.put(tableNames[0], new HCatSchema(ColumnHolder.hCattest1Cols)); + schemaMap.put(tableNames[1], new HCatSchema(ColumnHolder.hCattest2Cols)); + schemaMap.put(tableNames[2], new HCatSchema(ColumnHolder.hCattest3Cols)); + } + + private static class RunMS implements Runnable { + + @Override + public void run() { + try { + String warehouseConf = HiveConf.ConfVars.METASTOREWAREHOUSE.varname + "=" + + warehousedir.toString(); + HiveMetaStore.main(new String[]{"-v", "-p", msPort, "--hiveconf", warehouseConf}); + } catch (Throwable t) { + System.err.println("Exiting. Got exception from metastore: " + t.getMessage()); + } + } + + } + + /** + * Private class which holds all the data for the test cases + */ + private static class ColumnHolder { + + private static ArrayList hCattest1Cols = new ArrayList(); + private static ArrayList hCattest2Cols = new ArrayList(); + private static ArrayList hCattest3Cols = new ArrayList(); + + private static ArrayList partitionCols = new ArrayList(); + private static ArrayList test1Cols = new ArrayList(); + private static ArrayList test2Cols = new ArrayList(); + private static ArrayList test3Cols = new ArrayList(); + + private static HashMap> colMapping = new HashMap>(); + + static { + try { + FieldSchema keyCol = new FieldSchema("key", serdeConstants.STRING_TYPE_NAME, ""); + test1Cols.add(keyCol); + test2Cols.add(keyCol); + test3Cols.add(keyCol); + hCattest1Cols.add(HCatSchemaUtils.getHCatFieldSchema(keyCol)); + hCattest2Cols.add(HCatSchemaUtils.getHCatFieldSchema(keyCol)); + hCattest3Cols.add(HCatSchemaUtils.getHCatFieldSchema(keyCol)); + FieldSchema valueCol = new FieldSchema("value", serdeConstants.STRING_TYPE_NAME, ""); + test1Cols.add(valueCol); + test3Cols.add(valueCol); + hCattest1Cols.add(HCatSchemaUtils.getHCatFieldSchema(valueCol)); + hCattest3Cols.add(HCatSchemaUtils.getHCatFieldSchema(valueCol)); + FieldSchema extraCol = new FieldSchema("extra", serdeConstants.STRING_TYPE_NAME, ""); + test3Cols.add(extraCol); + hCattest3Cols.add(HCatSchemaUtils.getHCatFieldSchema(extraCol)); + colMapping.put("test1", test1Cols); + colMapping.put("test2", test2Cols); + colMapping.put("test3", test3Cols); + } catch (HCatException e) { + LOG.error("Error in setting up schema fields for the table", e); + throw new RuntimeException(e); + } + } + + static { + partitionCols.add(new FieldSchema("ds", serdeConstants.STRING_TYPE_NAME, "")); + partitionCols.add(new FieldSchema("cluster", serdeConstants.STRING_TYPE_NAME, "")); + } + } + + @BeforeClass + public static void setup() throws Exception { + String testDir = System.getProperty("test.data.dir", "./"); + testDir = testDir + "/test_multitable_" + Math.abs(new Random().nextLong()) + "/"; + workDir = new File(new File(testDir).getCanonicalPath()); + FileUtil.fullyDelete(workDir); + workDir.mkdirs(); + + warehousedir = new Path(workDir + "/warehouse"); + + // Run hive metastore server + t = new Thread(new RunMS()); + t.start(); + + // LocalJobRunner does not work with mapreduce OutputCommitter. So need + // to use MiniMRCluster. MAPREDUCE-2350 + Configuration conf = new Configuration(true); + conf.set("yarn.scheduler.capacity.root.queues", "default"); + conf.set("yarn.scheduler.capacity.root.default.capacity", "100"); + + FileSystem fs = FileSystem.get(conf); + System.setProperty("hadoop.log.dir", new File(workDir, "/logs").getAbsolutePath()); + mrCluster = new MiniMRCluster(1, fs.getUri().toString(), 1, null, null, + new JobConf(conf)); + mrConf = mrCluster.createJobConf(); + fs.mkdirs(warehousedir); + + initializeSetup(); + } + + private static void initializeSetup() throws Exception { + + hiveConf = new HiveConf(mrConf, TestHCatMultiOutputFormat.class); + hiveConf.set("hive.metastore.local", "false"); + hiveConf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:" + msPort); + hiveConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3); + hiveConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTFAILURERETRIES, 3); + hiveConf.set(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK.varname, + HCatSemanticAnalyzer.class.getName()); + hiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); + System.setProperty(HiveConf.ConfVars.PREEXECHOOKS.varname, " "); + System.setProperty(HiveConf.ConfVars.POSTEXECHOOKS.varname, " "); + + hiveConf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, warehousedir.toString()); + try { + hmsc = new HiveMetaStoreClient(hiveConf, null); + initalizeTables(); + } catch (Throwable e) { + LOG.error("Exception encountered while setting up testcase", e); + throw new Exception(e); + } finally { + hmsc.close(); + } + } + + private static void initalizeTables() throws Exception { + for (String table : tableNames) { + try { + if (hmsc.getTable(DATABASE, table) != null) { + hmsc.dropTable(DATABASE, table); + } + } catch (NoSuchObjectException ignored) { + } + } + for (int i = 0; i < tableNames.length; i++) { + createTable(tableNames[i], tablePerms[i]); + } + } + + private static void createTable(String tableName, String tablePerm) throws Exception { + Table tbl = new Table(); + tbl.setDbName(DATABASE); + tbl.setTableName(tableName); + StorageDescriptor sd = new StorageDescriptor(); + sd.setCols(ColumnHolder.colMapping.get(tableName)); + tbl.setSd(sd); + sd.setParameters(new HashMap()); + sd.setSerdeInfo(new SerDeInfo()); + sd.getSerdeInfo().setName(tbl.getTableName()); + sd.getSerdeInfo().setParameters(new HashMap()); + sd.setInputFormat(org.apache.hadoop.hive.ql.io.RCFileInputFormat.class.getName()); + sd.setOutputFormat(org.apache.hadoop.hive.ql.io.RCFileOutputFormat.class.getName()); + sd.getSerdeInfo().getParameters().put(serdeConstants.SERIALIZATION_FORMAT, "1"); + sd.getSerdeInfo().setSerializationLib( + org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe.class.getName()); + tbl.setPartitionKeys(ColumnHolder.partitionCols); + + hmsc.createTable(tbl); + FileSystem fs = FileSystem.get(mrConf); + fs.setPermission(new Path(warehousedir, tableName), new FsPermission(tablePerm)); + } + + @AfterClass + public static void tearDown() throws IOException { + FileUtil.fullyDelete(workDir); + FileSystem fs = FileSystem.get(mrConf); + if (fs.exists(warehousedir)) { + fs.delete(warehousedir, true); + } + if (mrCluster != null) { + mrCluster.shutdown(); + } + } + + /** + * Simple test case. + *
    + *
  1. Submits a mapred job which writes out one fixed line to each of the tables
  2. + *
  3. uses hive fetch task to read the data and see if it matches what was written
  4. + *
+ * + * @throws Exception if any error occurs + */ + @Test + public void testOutputFormat() throws Throwable { + HashMap partitionValues = new HashMap(); + partitionValues.put("ds", "1"); + partitionValues.put("cluster", "ag"); + ArrayList infoList = new ArrayList(); + infoList.add(OutputJobInfo.create("default", tableNames[0], partitionValues)); + infoList.add(OutputJobInfo.create("default", tableNames[1], partitionValues)); + infoList.add(OutputJobInfo.create("default", tableNames[2], partitionValues)); + + Job job = new Job(hiveConf, "SampleJob"); + + job.setMapperClass(MyMapper.class); + job.setInputFormatClass(TextInputFormat.class); + job.setOutputFormatClass(MultiOutputFormat.class); + job.setNumReduceTasks(0); + + JobConfigurer configurer = MultiOutputFormat.createConfigurer(job); + + for (int i = 0; i < tableNames.length; i++) { + configurer.addOutputFormat(tableNames[i], HCatOutputFormat.class, BytesWritable.class, + HCatRecord.class); + HCatOutputFormat.setOutput(configurer.getJob(tableNames[i]), infoList.get(i)); + HCatOutputFormat.setSchema(configurer.getJob(tableNames[i]), + schemaMap.get(tableNames[i])); + } + configurer.configure(); + + Path filePath = createInputFile(); + FileInputFormat.addInputPath(job, filePath); + Assert.assertTrue(job.waitForCompletion(true)); + + ArrayList outputs = new ArrayList(); + for (String tbl : tableNames) { + outputs.add(getTableData(tbl, "default").get(0)); + } + Assert.assertEquals("Comparing output of table " + + tableNames[0] + " is not correct", outputs.get(0), "a,a,1,ag"); + Assert.assertEquals("Comparing output of table " + + tableNames[1] + " is not correct", outputs.get(1), "a,1,ag"); + Assert.assertEquals("Comparing output of table " + + tableNames[2] + " is not correct", outputs.get(2), "a,a,extra,1,ag"); + + // Check permisssion on partition dirs and files created + for (int i = 0; i < tableNames.length; i++) { + Path partitionFile = new Path(warehousedir + "/" + tableNames[i] + + "/ds=1/cluster=ag/part-m-00000"); + FileSystem fs = partitionFile.getFileSystem(mrConf); + Assert.assertEquals("File permissions of table " + tableNames[i] + " is not correct", + fs.getFileStatus(partitionFile).getPermission(), + new FsPermission(tablePerms[i])); + Assert.assertEquals("File permissions of table " + tableNames[i] + " is not correct", + fs.getFileStatus(partitionFile.getParent()).getPermission(), + new FsPermission(tablePerms[i])); + Assert.assertEquals("File permissions of table " + tableNames[i] + " is not correct", + fs.getFileStatus(partitionFile.getParent().getParent()).getPermission(), + new FsPermission(tablePerms[i])); + + } + LOG.info("File permissions verified"); + } + + /** + * Create a input file for map + * + * @return absolute path of the file. + * @throws IOException if any error encountered + */ + private Path createInputFile() throws IOException { + Path f = new Path(workDir + "/MultiTableInput.txt"); + FileSystem fs = FileSystem.get(mrConf); + if (fs.exists(f)) { + fs.delete(f, true); + } + OutputStream out = fs.create(f); + for (int i = 0; i < 3; i++) { + out.write("a,a\n".getBytes()); + } + out.close(); + return f; + } + + /** + * Method to fetch table data + * + * @param table table name + * @param database database + * @return list of columns in comma seperated way + * @throws Exception if any error occurs + */ + private List getTableData(String table, String database) throws Exception { + HiveConf conf = new HiveConf(); + conf.addResource("hive-site.xml"); + ArrayList results = new ArrayList(); + ArrayList temp = new ArrayList(); + Hive hive = Hive.get(conf); + org.apache.hadoop.hive.ql.metadata.Table tbl = hive.getTable(database, table); + FetchWork work; + if (!tbl.getPartCols().isEmpty()) { + List partitions = hive.getPartitions(tbl); + List partDesc = new ArrayList(); + List partLocs = new ArrayList(); + for (Partition part : partitions) { + partLocs.add(part.getLocation()); + partDesc.add(Utilities.getPartitionDesc(part)); + } + work = new FetchWork(partLocs, partDesc, Utilities.getTableDesc(tbl)); + work.setLimit(100); + } else { + work = new FetchWork(tbl.getDataLocation().toString(), Utilities.getTableDesc(tbl)); + } + FetchTask task = new FetchTask(); + task.setWork(work); + task.initialize(conf, null, null); + task.fetch(temp); + for (String str : temp) { + results.add(str.replace("\t", ",")); + } + return results; + } + + private static class MyMapper extends + Mapper { + + private int i = 0; + + @Override + protected void map(LongWritable key, Text value, Context context) + throws IOException, InterruptedException { + HCatRecord record = null; + String[] splits = value.toString().split(","); + switch (i) { + case 0: + record = new DefaultHCatRecord(2); + record.set(0, splits[0]); + record.set(1, splits[1]); + break; + case 1: + record = new DefaultHCatRecord(1); + record.set(0, splits[0]); + break; + case 2: + record = new DefaultHCatRecord(3); + record.set(0, splits[0]); + record.set(1, splits[1]); + record.set(2, "extra"); + break; + default: + Assert.fail("This should not happen!!!!!"); + } + MultiOutputFormat.write(tableNames[i], null, record, context); + i++; + } + } +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatNonPartitioned.java hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatNonPartitioned.java new file mode 100644 index 0000000..36bb9b5 --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatNonPartitioned.java @@ -0,0 +1,140 @@ +/** + * 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.hcatalog.mapreduce; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.serde.serdeConstants; +import org.apache.hcatalog.common.ErrorType; +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.data.DefaultHCatRecord; +import org.apache.hcatalog.data.HCatRecord; +import org.apache.hcatalog.data.schema.HCatFieldSchema; +import org.apache.hcatalog.data.schema.HCatSchemaUtils; +import org.junit.BeforeClass; +import org.junit.Test; + +import static junit.framework.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.TestHCatNonPartitioned} instead + */ +public class TestHCatNonPartitioned extends HCatMapReduceTest { + + private static List writeRecords; + static List partitionColumns; + + @BeforeClass + public static void oneTimeSetUp() throws Exception { + + dbName = null; //test if null dbName works ("default" is used) + tableName = "testHCatNonPartitionedTable"; + + writeRecords = new ArrayList(); + + for (int i = 0; i < 20; i++) { + List objList = new ArrayList(); + + objList.add(i); + objList.add("strvalue" + i); + writeRecords.add(new DefaultHCatRecord(objList)); + } + + partitionColumns = new ArrayList(); + partitionColumns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("c1", serdeConstants.INT_TYPE_NAME, ""))); + partitionColumns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("c2", serdeConstants.STRING_TYPE_NAME, ""))); + } + + @Override + protected List getPartitionKeys() { + List fields = new ArrayList(); + //empty list, non partitioned + return fields; + } + + @Override + protected List getTableColumns() { + List fields = new ArrayList(); + fields.add(new FieldSchema("c1", serdeConstants.INT_TYPE_NAME, "")); + fields.add(new FieldSchema("c2", serdeConstants.STRING_TYPE_NAME, "")); + return fields; + } + + + @Test + public void testHCatNonPartitionedTable() throws Exception { + + Map partitionMap = new HashMap(); + runMRCreate(null, partitionColumns, writeRecords, 10, true); + + //Test for duplicate publish + IOException exc = null; + try { + runMRCreate(null, partitionColumns, writeRecords, 20, true); + } catch (IOException e) { + exc = e; + } + + assertTrue(exc != null); + assertTrue(exc instanceof HCatException); + assertEquals(ErrorType.ERROR_NON_EMPTY_TABLE, ((HCatException) exc).getErrorType()); + + //Test for publish with invalid partition key name + exc = null; + partitionMap.clear(); + partitionMap.put("px", "p1value2"); + + try { + runMRCreate(partitionMap, partitionColumns, writeRecords, 20, true); + } catch (IOException e) { + exc = e; + } + + assertTrue(exc != null); + assertTrue(exc instanceof HCatException); + assertEquals(ErrorType.ERROR_INVALID_PARTITION_VALUES, ((HCatException) exc).getErrorType()); + + //Read should get 10 rows + runMRRead(10); + + hiveReadTest(); + } + + //Test that data inserted through hcatoutputformat is readable from hive + private void hiveReadTest() throws Exception { + + String query = "select * from " + tableName; + int retCode = driver.run(query).getResponseCode(); + + if (retCode != 0) { + throw new Exception("Error " + retCode + " running query " + query); + } + + ArrayList res = new ArrayList(); + driver.getResults(res); + assertEquals(10, res.size()); + } +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatOutputFormat.java hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatOutputFormat.java new file mode 100644 index 0000000..53fa29e --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatOutputFormat.java @@ -0,0 +1,170 @@ +/** + * 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.hcatalog.mapreduce; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import junit.framework.TestCase; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.SerDeInfo; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.ql.io.RCFileInputFormat; +import org.apache.hadoop.hive.ql.io.RCFileOutputFormat; +import org.apache.hadoop.hive.serde.serdeConstants; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.OutputCommitter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.TestHCatOutputFormat} instead + */ +public class TestHCatOutputFormat extends TestCase { + + private static final Logger LOG = LoggerFactory.getLogger(TestHCatOutputFormat.class); + private HiveMetaStoreClient client; + private HiveConf hiveConf; + + private static final String dbName = "hcatOutputFormatTestDB"; + private static final String tblName = "hcatOutputFormatTestTable"; + + @Override + protected void setUp() throws Exception { + super.setUp(); + hiveConf = new HiveConf(this.getClass()); + + try { + client = new HiveMetaStoreClient(hiveConf, null); + + initTable(); + } catch (Throwable e) { + LOG.error("Unable to open the metastore", e); + throw new Exception(e); + } + } + + @Override + protected void tearDown() throws Exception { + try { + super.tearDown(); + client.dropTable(dbName, tblName); + client.dropDatabase(dbName); + + client.close(); + } catch (Throwable e) { + LOG.error("Unable to close metastore", e); + throw new Exception(e); + } + } + + private void initTable() throws Exception { + + try { + client.dropTable(dbName, tblName); + } catch (Exception e) { + } + try { + client.dropDatabase(dbName); + } catch (Exception e) { + } + client.createDatabase(new Database(dbName, "", null, null)); + assertNotNull((client.getDatabase(dbName).getLocationUri())); + + List fields = new ArrayList(); + fields.add(new FieldSchema("colname", serdeConstants.STRING_TYPE_NAME, "")); + + Table tbl = new Table(); + tbl.setDbName(dbName); + tbl.setTableName(tblName); + StorageDescriptor sd = new StorageDescriptor(); + sd.setCols(fields); + tbl.setSd(sd); + + //sd.setLocation("hdfs://tmp"); + sd.setInputFormat(RCFileInputFormat.class.getName()); + sd.setOutputFormat(RCFileOutputFormat.class.getName()); + sd.setParameters(new HashMap()); + sd.getParameters().put("test_param_1", "Use this for comments etc"); + //sd.setBucketCols(new ArrayList(2)); + //sd.getBucketCols().add("name"); + sd.setSerdeInfo(new SerDeInfo()); + sd.getSerdeInfo().setName(tbl.getTableName()); + sd.getSerdeInfo().setParameters(new HashMap()); + sd.getSerdeInfo().getParameters().put(serdeConstants.SERIALIZATION_FORMAT, "1"); + sd.getSerdeInfo().setSerializationLib( + org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe.class.getName()); + tbl.setPartitionKeys(fields); + + Map tableParams = new HashMap(); + tableParams.put("hcat.testarg", "testArgValue"); + + tbl.setParameters(tableParams); + + client.createTable(tbl); + Path tblPath = new Path(client.getTable(dbName, tblName).getSd().getLocation()); + assertTrue(tblPath.getFileSystem(hiveConf).mkdirs(new Path(tblPath, "colname=p1"))); + + } + + public void testSetOutput() throws Exception { + Configuration conf = new Configuration(); + Job job = new Job(conf, "test outputformat"); + + Map partitionValues = new HashMap(); + partitionValues.put("colname", "p1"); + //null server url means local mode + OutputJobInfo info = OutputJobInfo.create(dbName, tblName, partitionValues); + + HCatOutputFormat.setOutput(job, info); + OutputJobInfo jobInfo = HCatOutputFormat.getJobInfo(job); + + assertNotNull(jobInfo.getTableInfo()); + assertEquals(1, jobInfo.getPartitionValues().size()); + assertEquals("p1", jobInfo.getPartitionValues().get("colname")); + assertEquals(1, jobInfo.getTableInfo().getDataColumns().getFields().size()); + assertEquals("colname", jobInfo.getTableInfo().getDataColumns().getFields().get(0).getName()); + + publishTest(job); + } + + public void publishTest(Job job) throws Exception { + OutputCommitter committer = new FileOutputCommitterContainer(job, null); + committer.commitJob(job); + + Partition part = client.getPartition(dbName, tblName, Arrays.asList("p1")); + assertNotNull(part); + + StorerInfo storer = InternalUtil.extractStorerInfo(part.getSd(), part.getParameters()); + assertEquals(storer.getProperties().get("hcat.testarg"), "testArgValue"); + assertTrue(part.getSd().getLocation().indexOf("p1") != -1); + } +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatPartitionPublish.java hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatPartitionPublish.java new file mode 100644 index 0000000..2e34638 --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatPartitionPublish.java @@ -0,0 +1,259 @@ +/** + * 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.hcatalog.mapreduce; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import junit.framework.Assert; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.MetaStoreUtils; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.SerDeInfo; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.ql.io.RCFileInputFormat; +import org.apache.hadoop.hive.ql.io.RCFileOutputFormat; +import org.apache.hadoop.hive.serde.serdeConstants; +import org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.MiniMRCluster; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.Mapper; +import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; +import org.apache.hcatalog.NoExitSecurityManager; +import org.apache.hcatalog.cli.SemanticAnalysis.HCatSemanticAnalyzer; +import org.apache.hcatalog.data.DefaultHCatRecord; +import org.apache.hcatalog.data.HCatRecord; +import org.apache.hcatalog.data.schema.HCatFieldSchema; +import org.apache.hcatalog.data.schema.HCatSchema; +import org.apache.hcatalog.data.schema.HCatSchemaUtils; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.TestHCatPartitionPublish} instead + */ +public class TestHCatPartitionPublish { + private static Configuration mrConf = null; + private static FileSystem fs = null; + private static MiniMRCluster mrCluster = null; + private static boolean isServerRunning = false; + private static final int msPort = 20101; + private static HiveConf hcatConf; + private static HiveMetaStoreClient msc; + private static SecurityManager securityManager; + + @BeforeClass + public static void setup() throws Exception { + Configuration conf = new Configuration(true); + conf.set("yarn.scheduler.capacity.root.queues", "default"); + conf.set("yarn.scheduler.capacity.root.default.capacity", "100"); + + fs = FileSystem.get(conf); + System.setProperty("hadoop.log.dir", new File(fs.getWorkingDirectory() + .toString(), "/logs").getAbsolutePath()); + // LocalJobRunner does not work with mapreduce OutputCommitter. So need + // to use MiniMRCluster. MAPREDUCE-2350 + mrCluster = new MiniMRCluster(1, fs.getUri().toString(), 1, null, null, + new JobConf(conf)); + mrConf = mrCluster.createJobConf(); + + if (isServerRunning) { + return; + } + + MetaStoreUtils.startMetaStore(msPort, ShimLoader + .getHadoopThriftAuthBridge()); + isServerRunning = true; + securityManager = System.getSecurityManager(); + System.setSecurityManager(new NoExitSecurityManager()); + + hcatConf = new HiveConf(TestHCatPartitionPublish.class); + hcatConf.set("hive.metastore.local", "false"); + hcatConf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:" + + msPort); + hcatConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3); + hcatConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTFAILURERETRIES, 3); + hcatConf.set(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK.varname, + HCatSemanticAnalyzer.class.getName()); + hcatConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); + hcatConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); + hcatConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, + "false"); + msc = new HiveMetaStoreClient(hcatConf, null); + System.setProperty(HiveConf.ConfVars.PREEXECHOOKS.varname, " "); + System.setProperty(HiveConf.ConfVars.POSTEXECHOOKS.varname, " "); + } + + @AfterClass + public static void tearDown() throws IOException { + if (mrCluster != null) { + mrCluster.shutdown(); + } + System.setSecurityManager(securityManager); + isServerRunning = false; + } + + @Test + public void testPartitionPublish() throws Exception { + String dbName = "default"; + String tableName = "testHCatPartitionedTable"; + createTable(null, tableName); + + Map partitionMap = new HashMap(); + partitionMap.put("part1", "p1value1"); + partitionMap.put("part0", "p0value1"); + + ArrayList hcatTableColumns = new ArrayList(); + for (FieldSchema fs : getTableColumns()) { + hcatTableColumns.add(HCatSchemaUtils.getHCatFieldSchema(fs)); + } + + runMRCreateFail(dbName, tableName, partitionMap, hcatTableColumns); + List ptns = msc.listPartitionNames(dbName, tableName, + (short) 10); + Assert.assertEquals(0, ptns.size()); + Table table = msc.getTable(dbName, tableName); + Assert.assertTrue(table != null); + // Also make sure that the directory has been deleted in the table + // location. + Assert.assertFalse(fs.exists(new Path(table.getSd().getLocation() + + "/part1=p1value1/part0=p0value1"))); + } + + void runMRCreateFail( + String dbName, String tableName, Map partitionValues, + List columns) throws Exception { + + Job job = new Job(mrConf, "hcat mapreduce write fail test"); + job.setJarByClass(this.getClass()); + job.setMapperClass(TestHCatPartitionPublish.MapFail.class); + + // input/output settings + job.setInputFormatClass(TextInputFormat.class); + + Path path = new Path(fs.getWorkingDirectory(), + "mapred/testHCatMapReduceInput"); + // The write count does not matter, as the map will fail in its first + // call. + createInputFile(path, 5); + + TextInputFormat.setInputPaths(job, path); + job.setOutputFormatClass(HCatOutputFormat.class); + OutputJobInfo outputJobInfo = OutputJobInfo.create(dbName, tableName, + partitionValues); + HCatOutputFormat.setOutput(job, outputJobInfo); + + job.setMapOutputKeyClass(BytesWritable.class); + job.setMapOutputValueClass(DefaultHCatRecord.class); + + job.setNumReduceTasks(0); + + HCatOutputFormat.setSchema(job, new HCatSchema(columns)); + + boolean success = job.waitForCompletion(true); + Assert.assertTrue(success == false); + } + + private void createInputFile(Path path, int rowCount) throws IOException { + if (fs.exists(path)) { + fs.delete(path, true); + } + FSDataOutputStream os = fs.create(path); + for (int i = 0; i < rowCount; i++) { + os.writeChars(i + "\n"); + } + os.close(); + } + + public static class MapFail extends + Mapper { + + @Override + public void map(LongWritable key, Text value, Context context) + throws IOException, InterruptedException { + { + throw new IOException("Exception to mimic job failure."); + } + } + } + + private void createTable(String dbName, String tableName) throws Exception { + String databaseName = (dbName == null) ? MetaStoreUtils.DEFAULT_DATABASE_NAME + : dbName; + try { + msc.dropTable(databaseName, tableName); + } catch (Exception e) { + } // can fail with NoSuchObjectException + + Table tbl = new Table(); + tbl.setDbName(databaseName); + tbl.setTableName(tableName); + tbl.setTableType("MANAGED_TABLE"); + StorageDescriptor sd = new StorageDescriptor(); + sd.setCols(getTableColumns()); + tbl.setPartitionKeys(getPartitionKeys()); + tbl.setSd(sd); + sd.setBucketCols(new ArrayList(2)); + sd.setSerdeInfo(new SerDeInfo()); + sd.getSerdeInfo().setName(tbl.getTableName()); + sd.getSerdeInfo().setParameters(new HashMap()); + sd.getSerdeInfo().getParameters().put(serdeConstants.SERIALIZATION_FORMAT, "1"); + sd.getSerdeInfo().setSerializationLib(ColumnarSerDe.class.getName()); + sd.setInputFormat(RCFileInputFormat.class.getName()); + sd.setOutputFormat(RCFileOutputFormat.class.getName()); + + Map tableParams = new HashMap(); + tbl.setParameters(tableParams); + + msc.createTable(tbl); + } + + protected List getPartitionKeys() { + List fields = new ArrayList(); + // Defining partition names in unsorted order + fields.add(new FieldSchema("PaRT1", serdeConstants.STRING_TYPE_NAME, "")); + fields.add(new FieldSchema("part0", serdeConstants.STRING_TYPE_NAME, "")); + return fields; + } + + protected List getTableColumns() { + List fields = new ArrayList(); + fields.add(new FieldSchema("c1", serdeConstants.INT_TYPE_NAME, "")); + fields.add(new FieldSchema("c2", serdeConstants.STRING_TYPE_NAME, "")); + return fields; + } + +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatPartitioned.java hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatPartitioned.java new file mode 100644 index 0000000..d9b25d9 --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestHCatPartitioned.java @@ -0,0 +1,354 @@ +/** + * 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.hcatalog.mapreduce; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.serde.serdeConstants; +import org.apache.hcatalog.common.ErrorType; +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.data.DefaultHCatRecord; +import org.apache.hcatalog.data.HCatRecord; +import org.apache.hcatalog.data.schema.HCatFieldSchema; +import org.apache.hcatalog.data.schema.HCatSchema; +import org.apache.hcatalog.data.schema.HCatSchemaUtils; +import org.junit.BeforeClass; +import org.junit.Test; + +import static junit.framework.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.TestHCatPartitioned} instead + */ +public class TestHCatPartitioned extends HCatMapReduceTest { + + private static List writeRecords; + private static List partitionColumns; + + @BeforeClass + public static void oneTimeSetUp() throws Exception { + + tableName = "testHCatPartitionedTable"; + writeRecords = new ArrayList(); + + for (int i = 0; i < 20; i++) { + List objList = new ArrayList(); + + objList.add(i); + objList.add("strvalue" + i); + writeRecords.add(new DefaultHCatRecord(objList)); + } + + partitionColumns = new ArrayList(); + partitionColumns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("c1", serdeConstants.INT_TYPE_NAME, ""))); + partitionColumns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("c2", serdeConstants.STRING_TYPE_NAME, ""))); + } + + + @Override + protected List getPartitionKeys() { + List fields = new ArrayList(); + //Defining partition names in unsorted order + fields.add(new FieldSchema("PaRT1", serdeConstants.STRING_TYPE_NAME, "")); + fields.add(new FieldSchema("part0", serdeConstants.STRING_TYPE_NAME, "")); + return fields; + } + + @Override + protected List getTableColumns() { + List fields = new ArrayList(); + fields.add(new FieldSchema("c1", serdeConstants.INT_TYPE_NAME, "")); + fields.add(new FieldSchema("c2", serdeConstants.STRING_TYPE_NAME, "")); + return fields; + } + + + @Test + public void testHCatPartitionedTable() throws Exception { + + Map partitionMap = new HashMap(); + partitionMap.put("part1", "p1value1"); + partitionMap.put("part0", "p0value1"); + + runMRCreate(partitionMap, partitionColumns, writeRecords, 10, true); + + partitionMap.clear(); + partitionMap.put("PART1", "p1value2"); + partitionMap.put("PART0", "p0value2"); + + runMRCreate(partitionMap, partitionColumns, writeRecords, 20, true); + + //Test for duplicate publish + IOException exc = null; + try { + runMRCreate(partitionMap, partitionColumns, writeRecords, 20, true); + } catch (IOException e) { + exc = e; + } + + assertTrue(exc != null); + assertTrue(exc instanceof HCatException); + assertEquals(ErrorType.ERROR_DUPLICATE_PARTITION, ((HCatException) exc).getErrorType()); + + //Test for publish with invalid partition key name + exc = null; + partitionMap.clear(); + partitionMap.put("px1", "p1value2"); + partitionMap.put("px0", "p0value2"); + + try { + runMRCreate(partitionMap, partitionColumns, writeRecords, 20, true); + } catch (IOException e) { + exc = e; + } + + assertTrue(exc != null); + assertTrue(exc instanceof HCatException); + assertEquals(ErrorType.ERROR_MISSING_PARTITION_KEY, ((HCatException) exc).getErrorType()); + + //Test for publish with missing partition key values + exc = null; + partitionMap.clear(); + partitionMap.put("px", "p1value2"); + + try { + runMRCreate(partitionMap, partitionColumns, writeRecords, 20, true); + } catch (IOException e) { + exc = e; + } + + assertTrue(exc != null); + assertTrue(exc instanceof HCatException); + assertEquals(ErrorType.ERROR_INVALID_PARTITION_VALUES, ((HCatException) exc).getErrorType()); + + + //Test for null partition value map + exc = null; + try { + runMRCreate(null, partitionColumns, writeRecords, 20, false); + } catch (IOException e) { + exc = e; + } + + assertTrue(exc == null); +// assertTrue(exc instanceof HCatException); +// assertEquals(ErrorType.ERROR_PUBLISHING_PARTITION, ((HCatException) exc).getErrorType()); + // With Dynamic partitioning, this isn't an error that the keyValues specified didn't values + + //Read should get 10 + 20 rows + runMRRead(30); + + //Read with partition filter + runMRRead(10, "part1 = \"p1value1\""); + runMRRead(20, "part1 = \"p1value2\""); + runMRRead(30, "part1 = \"p1value1\" or part1 = \"p1value2\""); + runMRRead(10, "part0 = \"p0value1\""); + runMRRead(20, "part0 = \"p0value2\""); + runMRRead(30, "part0 = \"p0value1\" or part0 = \"p0value2\""); + + tableSchemaTest(); + columnOrderChangeTest(); + hiveReadTest(); + } + + + //test that new columns gets added to table schema + private void tableSchemaTest() throws Exception { + + HCatSchema tableSchema = getTableSchema(); + + assertEquals(4, tableSchema.getFields().size()); + + //Update partition schema to have 3 fields + partitionColumns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("c3", serdeConstants.STRING_TYPE_NAME, ""))); + + writeRecords = new ArrayList(); + + for (int i = 0; i < 20; i++) { + List objList = new ArrayList(); + + objList.add(i); + objList.add("strvalue" + i); + objList.add("str2value" + i); + + writeRecords.add(new DefaultHCatRecord(objList)); + } + + Map partitionMap = new HashMap(); + partitionMap.put("part1", "p1value5"); + partitionMap.put("part0", "p0value5"); + + runMRCreate(partitionMap, partitionColumns, writeRecords, 10, true); + + tableSchema = getTableSchema(); + + //assert that c3 has got added to table schema + assertEquals(5, tableSchema.getFields().size()); + assertEquals("c1", tableSchema.getFields().get(0).getName()); + assertEquals("c2", tableSchema.getFields().get(1).getName()); + assertEquals("c3", tableSchema.getFields().get(2).getName()); + assertEquals("part1", tableSchema.getFields().get(3).getName()); + assertEquals("part0", tableSchema.getFields().get(4).getName()); + + //Test that changing column data type fails + partitionMap.clear(); + partitionMap.put("part1", "p1value6"); + partitionMap.put("part0", "p0value6"); + + partitionColumns = new ArrayList(); + partitionColumns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("c1", serdeConstants.INT_TYPE_NAME, ""))); + partitionColumns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("c2", serdeConstants.INT_TYPE_NAME, ""))); + + IOException exc = null; + try { + runMRCreate(partitionMap, partitionColumns, writeRecords, 20, true); + } catch (IOException e) { + exc = e; + } + + assertTrue(exc != null); + assertTrue(exc instanceof HCatException); + assertEquals(ErrorType.ERROR_SCHEMA_TYPE_MISMATCH, ((HCatException) exc).getErrorType()); + + //Test that partition key is not allowed in data + partitionColumns = new ArrayList(); + partitionColumns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("c1", serdeConstants.INT_TYPE_NAME, ""))); + partitionColumns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("c2", serdeConstants.STRING_TYPE_NAME, ""))); + partitionColumns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("c3", serdeConstants.STRING_TYPE_NAME, ""))); + partitionColumns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("part1", serdeConstants.STRING_TYPE_NAME, ""))); + + List recordsContainingPartitionCols = new ArrayList(20); + for (int i = 0; i < 20; i++) { + List objList = new ArrayList(); + + objList.add(i); + objList.add("c2value" + i); + objList.add("c3value" + i); + objList.add("p1value6"); + + recordsContainingPartitionCols.add(new DefaultHCatRecord(objList)); + } + + exc = null; + try { + runMRCreate(partitionMap, partitionColumns, recordsContainingPartitionCols, 20, true); + } catch (IOException e) { + exc = e; + } + + List records = runMRRead(20, "part1 = \"p1value6\""); + assertEquals(20, records.size()); + records = runMRRead(20, "part0 = \"p0value6\""); + assertEquals(20, records.size()); + Integer i = 0; + for (HCatRecord rec : records) { + assertEquals(5, rec.size()); + assertTrue(rec.get(0).equals(i)); + assertTrue(rec.get(1).equals("c2value" + i)); + assertTrue(rec.get(2).equals("c3value" + i)); + assertTrue(rec.get(3).equals("p1value6")); + assertTrue(rec.get(4).equals("p0value6")); + i++; + } + } + + //check behavior while change the order of columns + private void columnOrderChangeTest() throws Exception { + + HCatSchema tableSchema = getTableSchema(); + + assertEquals(5, tableSchema.getFields().size()); + + partitionColumns = new ArrayList(); + partitionColumns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("c1", serdeConstants.INT_TYPE_NAME, ""))); + partitionColumns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("c3", serdeConstants.STRING_TYPE_NAME, ""))); + partitionColumns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("c2", serdeConstants.STRING_TYPE_NAME, ""))); + + + writeRecords = new ArrayList(); + + for (int i = 0; i < 10; i++) { + List objList = new ArrayList(); + + objList.add(i); + objList.add("co strvalue" + i); + objList.add("co str2value" + i); + + writeRecords.add(new DefaultHCatRecord(objList)); + } + + Map partitionMap = new HashMap(); + partitionMap.put("part1", "p1value8"); + partitionMap.put("part0", "p0value8"); + + Exception exc = null; + try { + runMRCreate(partitionMap, partitionColumns, writeRecords, 10, true); + } catch (IOException e) { + exc = e; + } + + assertTrue(exc != null); + assertTrue(exc instanceof HCatException); + assertEquals(ErrorType.ERROR_SCHEMA_COLUMN_MISMATCH, ((HCatException) exc).getErrorType()); + + + partitionColumns = new ArrayList(); + partitionColumns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("c1", serdeConstants.INT_TYPE_NAME, ""))); + partitionColumns.add(HCatSchemaUtils.getHCatFieldSchema(new FieldSchema("c2", serdeConstants.STRING_TYPE_NAME, ""))); + + writeRecords = new ArrayList(); + + for (int i = 0; i < 10; i++) { + List objList = new ArrayList(); + + objList.add(i); + objList.add("co strvalue" + i); + + writeRecords.add(new DefaultHCatRecord(objList)); + } + + runMRCreate(partitionMap, partitionColumns, writeRecords, 10, true); + + //Read should get 10 + 20 + 10 + 10 + 20 rows + runMRRead(70); + } + + //Test that data inserted through hcatoutputformat is readable from hive + private void hiveReadTest() throws Exception { + + String query = "select * from " + tableName; + int retCode = driver.run(query).getResponseCode(); + + if (retCode != 0) { + throw new Exception("Error " + retCode + " running query " + query); + } + + ArrayList res = new ArrayList(); + driver.getResults(res); + assertEquals(70, res.size()); + } +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestInputJobInfo.java hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestInputJobInfo.java new file mode 100644 index 0000000..2867280 --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestInputJobInfo.java @@ -0,0 +1,51 @@ +/** + * 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.hcatalog.mapreduce; + +import java.util.Properties; + +import junit.framework.Assert; + +import org.junit.Test; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.TestInputJobInfo} instead + */ +public class TestInputJobInfo extends HCatBaseTest { + + @Test + public void test4ArgCreate() throws Exception { + Properties p = new Properties(); + p.setProperty("key", "value"); + InputJobInfo jobInfo = InputJobInfo.create("Db", "Table", "Filter", p); + Assert.assertEquals("Db", jobInfo.getDatabaseName()); + Assert.assertEquals("Table", jobInfo.getTableName()); + Assert.assertEquals("Filter", jobInfo.getFilter()); + Assert.assertEquals("value", jobInfo.getProperties().getProperty("key")); + } + + @Test + public void test3ArgCreate() throws Exception { + InputJobInfo jobInfo = InputJobInfo.create("Db", "Table", "Filter"); + Assert.assertEquals("Db", jobInfo.getDatabaseName()); + Assert.assertEquals("Table", jobInfo.getTableName()); + Assert.assertEquals("Filter", jobInfo.getFilter()); + Assert.assertEquals(0, jobInfo.getProperties().size()); + } +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestMultiOutputFormat.java hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestMultiOutputFormat.java new file mode 100644 index 0000000..c05a7f0 --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestMultiOutputFormat.java @@ -0,0 +1,337 @@ +/** + * 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.hcatalog.mapreduce; + +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.net.URI; +import java.util.Arrays; +import java.util.List; +import java.util.Random; +import java.util.StringTokenizer; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.filecache.DistributedCache; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.SequenceFile; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hadoop.mapred.MiniMRCluster; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.Mapper; +import org.apache.hadoop.mapreduce.OutputCommitter; +import org.apache.hadoop.mapreduce.Reducer; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; +import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; +import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; +import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat; +import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat; +import org.apache.hcatalog.mapreduce.MultiOutputFormat.JobConfigurer; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.TestMultiOutputFormat} instead + */ +public class TestMultiOutputFormat { + + private static final Logger LOG = LoggerFactory.getLogger(TestMultiOutputFormat.class); + private static File workDir; + private static Configuration mrConf = null; + private static FileSystem fs = null; + private static MiniMRCluster mrCluster = null; + + @BeforeClass + public static void setup() throws IOException { + createWorkDir(); + Configuration conf = new Configuration(true); + conf.set("yarn.scheduler.capacity.root.queues", "default"); + conf.set("yarn.scheduler.capacity.root.default.capacity", "100"); + + fs = FileSystem.get(conf); + System.setProperty("hadoop.log.dir", new File(workDir, "/logs").getAbsolutePath()); + // LocalJobRunner does not work with mapreduce OutputCommitter. So need + // to use MiniMRCluster. MAPREDUCE-2350 + mrCluster = new MiniMRCluster(1, fs.getUri().toString(), 1, null, null, + new JobConf(conf)); + mrConf = mrCluster.createJobConf(); + } + + private static void createWorkDir() throws IOException { + String testDir = System.getProperty("test.data.dir", "./"); + testDir = testDir + "/test_multiout_" + Math.abs(new Random().nextLong()) + "/"; + workDir = new File(new File(testDir).getCanonicalPath()); + FileUtil.fullyDelete(workDir); + workDir.mkdirs(); + } + + @AfterClass + public static void tearDown() throws IOException { + if (mrCluster != null) { + mrCluster.shutdown(); + } + FileUtil.fullyDelete(workDir); + } + + /** + * A test job that reads a input file and outputs each word and the index of + * the word encountered to a text file and sequence file with different key + * values. + */ + @Test + public void testMultiOutputFormatWithoutReduce() throws Throwable { + Job job = new Job(mrConf, "MultiOutNoReduce"); + job.setMapperClass(MultiOutWordIndexMapper.class); + job.setJarByClass(this.getClass()); + job.setInputFormatClass(TextInputFormat.class); + job.setOutputFormatClass(MultiOutputFormat.class); + job.setNumReduceTasks(0); + + JobConfigurer configurer = MultiOutputFormat.createConfigurer(job); + configurer.addOutputFormat("out1", TextOutputFormat.class, IntWritable.class, Text.class); + configurer.addOutputFormat("out2", SequenceFileOutputFormat.class, Text.class, + IntWritable.class); + Path outDir = new Path(workDir.getPath(), job.getJobName()); + FileOutputFormat.setOutputPath(configurer.getJob("out1"), new Path(outDir, "out1")); + FileOutputFormat.setOutputPath(configurer.getJob("out2"), new Path(outDir, "out2")); + + String fileContent = "Hello World"; + String inputFile = createInputFile(fileContent); + FileInputFormat.setInputPaths(job, new Path(inputFile)); + + //Test for merging of configs + DistributedCache.addFileToClassPath(new Path(inputFile), job.getConfiguration(), fs); + String dummyFile = createInputFile("dummy file"); + DistributedCache.addFileToClassPath(new Path(dummyFile), configurer.getJob("out1") + .getConfiguration(), fs); + // duplicate of the value. Merging should remove duplicates + DistributedCache.addFileToClassPath(new Path(inputFile), configurer.getJob("out2") + .getConfiguration(), fs); + + configurer.configure(); + + // Verify if the configs are merged + Path[] fileClassPaths = DistributedCache.getFileClassPaths(job.getConfiguration()); + List fileClassPathsList = Arrays.asList(fileClassPaths); + Assert.assertTrue(fileClassPathsList.contains(new Path(inputFile))); + Assert.assertTrue(fileClassPathsList.contains(new Path(dummyFile))); + + URI[] cacheFiles = DistributedCache.getCacheFiles(job.getConfiguration()); + List cacheFilesList = Arrays.asList(cacheFiles); + Assert.assertTrue(cacheFilesList.contains(new Path(inputFile).makeQualified(fs).toUri())); + Assert.assertTrue(cacheFilesList.contains(new Path(dummyFile).makeQualified(fs).toUri())); + + Assert.assertTrue(job.waitForCompletion(true)); + + Path textOutPath = new Path(outDir, "out1/part-m-00000"); + String[] textOutput = readFully(textOutPath).split("\n"); + Path seqOutPath = new Path(outDir, "out2/part-m-00000"); + SequenceFile.Reader reader = new SequenceFile.Reader(fs, seqOutPath, mrConf); + Text key = new Text(); + IntWritable value = new IntWritable(); + String[] words = fileContent.split(" "); + Assert.assertEquals(words.length, textOutput.length); + LOG.info("Verifying file contents"); + for (int i = 0; i < words.length; i++) { + Assert.assertEquals((i + 1) + "\t" + words[i], textOutput[i]); + reader.next(key, value); + Assert.assertEquals(words[i], key.toString()); + Assert.assertEquals((i + 1), value.get()); + } + Assert.assertFalse(reader.next(key, value)); + } + + /** + * A word count test job that reads a input file and outputs the count of + * words to a text file and sequence file with different key values. + */ + @Test + public void testMultiOutputFormatWithReduce() throws Throwable { + Job job = new Job(mrConf, "MultiOutWithReduce"); + + job.setMapperClass(WordCountMapper.class); + job.setReducerClass(MultiOutWordCountReducer.class); + job.setJarByClass(this.getClass()); + job.setInputFormatClass(TextInputFormat.class); + job.setOutputFormatClass(MultiOutputFormat.class); + job.setMapOutputKeyClass(Text.class); + job.setMapOutputValueClass(IntWritable.class); + + JobConfigurer configurer = MultiOutputFormat.createConfigurer(job); + + configurer.addOutputFormat("out1", TextOutputFormat.class, IntWritable.class, Text.class); + configurer.addOutputFormat("out2", SequenceFileOutputFormat.class, Text.class, + IntWritable.class); + configurer.addOutputFormat("out3", NullOutputFormat.class, Text.class, + IntWritable.class); + Path outDir = new Path(workDir.getPath(), job.getJobName()); + FileOutputFormat.setOutputPath(configurer.getJob("out1"), new Path(outDir, "out1")); + FileOutputFormat.setOutputPath(configurer.getJob("out2"), new Path(outDir, "out2")); + + configurer.configure(); + + String fileContent = "Hello World Hello World World"; + String inputFile = createInputFile(fileContent); + FileInputFormat.setInputPaths(job, new Path(inputFile)); + + Assert.assertTrue(job.waitForCompletion(true)); + + Path textOutPath = new Path(outDir, "out1/part-r-00000"); + String[] textOutput = readFully(textOutPath).split("\n"); + Path seqOutPath = new Path(outDir, "out2/part-r-00000"); + SequenceFile.Reader reader = new SequenceFile.Reader(fs, seqOutPath, mrConf); + Text key = new Text(); + IntWritable value = new IntWritable(); + String[] words = "Hello World".split(" "); + Assert.assertEquals(words.length, textOutput.length); + for (int i = 0; i < words.length; i++) { + Assert.assertEquals((i + 2) + "\t" + words[i], textOutput[i]); + reader.next(key, value); + Assert.assertEquals(words[i], key.toString()); + Assert.assertEquals((i + 2), value.get()); + } + Assert.assertFalse(reader.next(key, value)); + + } + + + /** + * Create a file for map input + * + * @return absolute path of the file. + * @throws IOException if any error encountered + */ + private String createInputFile(String content) throws IOException { + File f = File.createTempFile("input", "txt"); + FileWriter writer = new FileWriter(f); + writer.write(content); + writer.close(); + return f.getAbsolutePath(); + } + + private String readFully(Path file) throws IOException { + FSDataInputStream in = fs.open(file); + byte[] b = new byte[in.available()]; + in.readFully(b); + in.close(); + return new String(b); + } + + private static class MultiOutWordIndexMapper extends + Mapper { + + private IntWritable index = new IntWritable(1); + private Text word = new Text(); + + @Override + protected void map(LongWritable key, Text value, Context context) + throws IOException, InterruptedException { + StringTokenizer itr = new StringTokenizer(value.toString()); + while (itr.hasMoreTokens()) { + word.set(itr.nextToken()); + MultiOutputFormat.write("out1", index, word, context); + MultiOutputFormat.write("out2", word, index, context); + index.set(index.get() + 1); + } + } + } + + private static class WordCountMapper extends + Mapper { + + private final static IntWritable one = new IntWritable(1); + private Text word = new Text(); + + @Override + protected void map(LongWritable key, Text value, Context context) + throws IOException, InterruptedException { + StringTokenizer itr = new StringTokenizer(value.toString()); + while (itr.hasMoreTokens()) { + word.set(itr.nextToken()); + context.write(word, one); + } + } + } + + private static class MultiOutWordCountReducer extends + Reducer { + + private IntWritable count = new IntWritable(); + + @Override + protected void reduce(Text word, Iterable values, Context context) + throws IOException, InterruptedException { + int sum = 0; + for (IntWritable val : values) { + sum += val.get(); + } + count.set(sum); + MultiOutputFormat.write("out1", count, word, context); + MultiOutputFormat.write("out2", word, count, context); + MultiOutputFormat.write("out3", word, count, context); + } + } + + private static class NullOutputFormat extends + org.apache.hadoop.mapreduce.lib.output.NullOutputFormat { + + @Override + public OutputCommitter getOutputCommitter(TaskAttemptContext context) { + return new OutputCommitter() { + public void abortTask(TaskAttemptContext taskContext) { + } + + public void cleanupJob(JobContext jobContext) { + } + + public void commitJob(JobContext jobContext) { + } + + public void commitTask(TaskAttemptContext taskContext) { + Assert.fail("needsTaskCommit is false but commitTask was called"); + } + + public boolean needsTaskCommit(TaskAttemptContext taskContext) { + return false; + } + + public void setupJob(JobContext jobContext) { + } + + public void setupTask(TaskAttemptContext taskContext) { + } + }; + } + } + +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestPassProperties.java hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestPassProperties.java new file mode 100644 index 0000000..9b30576 --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestPassProperties.java @@ -0,0 +1,146 @@ +/** + * 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.hcatalog.mapreduce; + +import static org.junit.Assert.assertTrue; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.cli.CliSessionState; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.MetaStoreUtils; +import org.apache.hadoop.hive.ql.Driver; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.Mapper; +import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; +import org.apache.hcatalog.HcatTestUtils; +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.data.DefaultHCatRecord; +import org.apache.hcatalog.data.schema.HCatFieldSchema; +import org.apache.hcatalog.data.schema.HCatSchema; +import org.apache.pig.ExecType; +import org.apache.pig.PigServer; +import org.junit.Test; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.TestPassProperties} instead + */ +public class TestPassProperties { + private static final String TEST_DATA_DIR = System.getProperty("user.dir") + + "/build/test/data/" + TestSequenceFileReadWrite.class.getCanonicalName(); + private static final String TEST_WAREHOUSE_DIR = TEST_DATA_DIR + "/warehouse"; + private static final String INPUT_FILE_NAME = TEST_DATA_DIR + "/input.data"; + + private static Driver driver; + private static PigServer server; + private static String[] input; + private static HiveConf hiveConf; + + public void Initialize() throws Exception { + hiveConf = new HiveConf(this.getClass()); + hiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); + hiveConf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, TEST_WAREHOUSE_DIR); + driver = new Driver(hiveConf); + SessionState.start(new CliSessionState(hiveConf)); + + new File(TEST_WAREHOUSE_DIR).mkdirs(); + + int numRows = 3; + input = new String[numRows]; + for (int i = 0; i < numRows; i++) { + String col1 = "a" + i; + String col2 = "b" + i; + input[i] = i + "," + col1 + "," + col2; + } + HcatTestUtils.createTestDataFile(INPUT_FILE_NAME, input); + server = new PigServer(ExecType.LOCAL); + } + + @Test + public void testSequenceTableWriteReadMR() throws Exception { + Initialize(); + String createTable = "CREATE TABLE bad_props_table(a0 int, a1 String, a2 String) STORED AS SEQUENCEFILE"; + driver.run("drop table bad_props_table"); + int retCode1 = driver.run(createTable).getResponseCode(); + assertTrue(retCode1 == 0); + + boolean caughtException = false; + try { + Configuration conf = new Configuration(); + conf.set("hive.metastore.uris", "thrift://no.such.machine:10888"); + conf.set("hive.metastore.local", "false"); + Job job = new Job(conf, "Write-hcat-seq-table"); + job.setJarByClass(TestSequenceFileReadWrite.class); + + job.setMapperClass(Map.class); + job.setOutputKeyClass(NullWritable.class); + job.setOutputValueClass(DefaultHCatRecord.class); + job.setInputFormatClass(TextInputFormat.class); + TextInputFormat.setInputPaths(job, INPUT_FILE_NAME); + + HCatOutputFormat.setOutput(job, OutputJobInfo.create( + MetaStoreUtils.DEFAULT_DATABASE_NAME, "bad_props_table", null)); + job.setOutputFormatClass(HCatOutputFormat.class); + HCatOutputFormat.setSchema(job, getSchema()); + job.setNumReduceTasks(0); + assertTrue(job.waitForCompletion(true)); + new FileOutputCommitterContainer(job, null).cleanupJob(job); + } catch (Exception e) { + caughtException = true; + assertTrue(e.getMessage().contains( + "Could not connect to meta store using any of the URIs provided")); + } + assertTrue(caughtException); + } + + public static class Map extends Mapper { + + public void map(LongWritable key, Text value, Context context) throws IOException, InterruptedException { + String[] cols = value.toString().split(","); + DefaultHCatRecord record = new DefaultHCatRecord(3); + record.set(0, Integer.parseInt(cols[0])); + record.set(1, cols[1]); + record.set(2, cols[2]); + context.write(NullWritable.get(), record); + } + } + + private HCatSchema getSchema() throws HCatException { + HCatSchema schema = new HCatSchema(new ArrayList()); + schema.append(new HCatFieldSchema("a0", HCatFieldSchema.Type.INT, + "")); + schema.append(new HCatFieldSchema("a1", + HCatFieldSchema.Type.STRING, "")); + schema.append(new HCatFieldSchema("a2", + HCatFieldSchema.Type.STRING, "")); + return schema; + } + + +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestSequenceFileReadWrite.java hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestSequenceFileReadWrite.java new file mode 100644 index 0000000..2cf3bdc --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/mapreduce/TestSequenceFileReadWrite.java @@ -0,0 +1,268 @@ +/** + * 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.hcatalog.mapreduce; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; + +import junit.framework.TestCase; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.cli.CliSessionState; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.MetaStoreUtils; +import org.apache.hadoop.hive.ql.Driver; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.Mapper; +import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; +import org.apache.hcatalog.HcatTestUtils; +import org.apache.hcatalog.common.HCatConstants; +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.common.HCatUtil; +import org.apache.hcatalog.data.DefaultHCatRecord; +import org.apache.hcatalog.data.schema.HCatFieldSchema; +import org.apache.hcatalog.data.schema.HCatSchema; +import org.apache.pig.ExecType; +import org.apache.pig.PigServer; +import org.apache.pig.data.Tuple; +import org.junit.Test; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.mapreduce.TestSequenceFileReadWrite} instead + */ +public class TestSequenceFileReadWrite extends TestCase { + private static final String TEST_DATA_DIR = System.getProperty("user.dir") + + "/build/test/data/" + TestSequenceFileReadWrite.class.getCanonicalName(); + private static final String TEST_WAREHOUSE_DIR = TEST_DATA_DIR + "/warehouse"; + private static final String INPUT_FILE_NAME = TEST_DATA_DIR + "/input.data"; + + private static Driver driver; + private static PigServer server; + private static String[] input; + private static HiveConf hiveConf; + + public void Initialize() throws Exception { + hiveConf = new HiveConf(this.getClass()); + hiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); + hiveConf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, TEST_WAREHOUSE_DIR); + driver = new Driver(hiveConf); + SessionState.start(new CliSessionState(hiveConf)); + + new File(TEST_WAREHOUSE_DIR).mkdirs(); + + int numRows = 3; + input = new String[numRows]; + for (int i = 0; i < numRows; i++) { + String col1 = "a" + i; + String col2 = "b" + i; + input[i] = i + "," + col1 + "," + col2; + } + HcatTestUtils.createTestDataFile(INPUT_FILE_NAME, input); + server = new PigServer(ExecType.LOCAL); + } + + @Test + public void testSequenceTableWriteRead() throws Exception { + Initialize(); + String createTable = "CREATE TABLE demo_table(a0 int, a1 String, a2 String) STORED AS SEQUENCEFILE"; + driver.run("drop table demo_table"); + int retCode1 = driver.run(createTable).getResponseCode(); + assertTrue(retCode1 == 0); + + server.setBatchOn(); + server.registerQuery("A = load '" + + INPUT_FILE_NAME + + "' using PigStorage(',') as (a0:int,a1:chararray,a2:chararray);"); + server.registerQuery("store A into 'demo_table' using org.apache.hcatalog.pig.HCatStorer();"); + server.executeBatch(); + + server.registerQuery("B = load 'demo_table' using org.apache.hcatalog.pig.HCatLoader();"); + Iterator XIter = server.openIterator("B"); + int numTuplesRead = 0; + while (XIter.hasNext()) { + Tuple t = XIter.next(); + assertEquals(3, t.size()); + assertEquals(t.get(0).toString(), "" + numTuplesRead); + assertEquals(t.get(1).toString(), "a" + numTuplesRead); + assertEquals(t.get(2).toString(), "b" + numTuplesRead); + numTuplesRead++; + } + assertEquals(input.length, numTuplesRead); + } + + @Test + public void testTextTableWriteRead() throws Exception { + Initialize(); + String createTable = "CREATE TABLE demo_table_1(a0 int, a1 String, a2 String) STORED AS TEXTFILE"; + driver.run("drop table demo_table_1"); + int retCode1 = driver.run(createTable).getResponseCode(); + assertTrue(retCode1 == 0); + + server.setBatchOn(); + server.registerQuery("A = load '" + + INPUT_FILE_NAME + + "' using PigStorage(',') as (a0:int,a1:chararray,a2:chararray);"); + server.registerQuery("store A into 'demo_table_1' using org.apache.hcatalog.pig.HCatStorer();"); + server.executeBatch(); + + server.registerQuery("B = load 'demo_table_1' using org.apache.hcatalog.pig.HCatLoader();"); + Iterator XIter = server.openIterator("B"); + int numTuplesRead = 0; + while (XIter.hasNext()) { + Tuple t = XIter.next(); + assertEquals(3, t.size()); + assertEquals(t.get(0).toString(), "" + numTuplesRead); + assertEquals(t.get(1).toString(), "a" + numTuplesRead); + assertEquals(t.get(2).toString(), "b" + numTuplesRead); + numTuplesRead++; + } + assertEquals(input.length, numTuplesRead); + } + + @Test + public void testSequenceTableWriteReadMR() throws Exception { + Initialize(); + String createTable = "CREATE TABLE demo_table_2(a0 int, a1 String, a2 String) STORED AS SEQUENCEFILE"; + driver.run("drop table demo_table_2"); + int retCode1 = driver.run(createTable).getResponseCode(); + assertTrue(retCode1 == 0); + + Configuration conf = new Configuration(); + conf.set(HCatConstants.HCAT_KEY_HIVE_CONF, + HCatUtil.serialize(hiveConf.getAllProperties())); + Job job = new Job(conf, "Write-hcat-seq-table"); + job.setJarByClass(TestSequenceFileReadWrite.class); + + job.setMapperClass(Map.class); + job.setOutputKeyClass(NullWritable.class); + job.setOutputValueClass(DefaultHCatRecord.class); + job.setInputFormatClass(TextInputFormat.class); + TextInputFormat.setInputPaths(job, INPUT_FILE_NAME); + + HCatOutputFormat.setOutput(job, OutputJobInfo.create( + MetaStoreUtils.DEFAULT_DATABASE_NAME, "demo_table_2", null)); + job.setOutputFormatClass(HCatOutputFormat.class); + HCatOutputFormat.setSchema(job, getSchema()); + job.setNumReduceTasks(0); + assertTrue(job.waitForCompletion(true)); + if (!HCatUtil.isHadoop23()) { + new FileOutputCommitterContainer(job, null).commitJob(job); + } + assertTrue(job.isSuccessful()); + + server.setBatchOn(); + server.registerQuery("C = load 'default.demo_table_2' using org.apache.hcatalog.pig.HCatLoader();"); + server.executeBatch(); + Iterator XIter = server.openIterator("C"); + int numTuplesRead = 0; + while (XIter.hasNext()) { + Tuple t = XIter.next(); + assertEquals(3, t.size()); + assertEquals(t.get(0).toString(), "" + numTuplesRead); + assertEquals(t.get(1).toString(), "a" + numTuplesRead); + assertEquals(t.get(2).toString(), "b" + numTuplesRead); + numTuplesRead++; + } + assertEquals(input.length, numTuplesRead); + } + + @Test + public void testTextTableWriteReadMR() throws Exception { + Initialize(); + String createTable = "CREATE TABLE demo_table_3(a0 int, a1 String, a2 String) STORED AS TEXTFILE"; + driver.run("drop table demo_table_3"); + int retCode1 = driver.run(createTable).getResponseCode(); + assertTrue(retCode1 == 0); + + Configuration conf = new Configuration(); + conf.set(HCatConstants.HCAT_KEY_HIVE_CONF, + HCatUtil.serialize(hiveConf.getAllProperties())); + Job job = new Job(conf, "Write-hcat-text-table"); + job.setJarByClass(TestSequenceFileReadWrite.class); + + job.setMapperClass(Map.class); + job.setOutputKeyClass(NullWritable.class); + job.setOutputValueClass(DefaultHCatRecord.class); + job.setInputFormatClass(TextInputFormat.class); + job.setNumReduceTasks(0); + TextInputFormat.setInputPaths(job, INPUT_FILE_NAME); + + HCatOutputFormat.setOutput(job, OutputJobInfo.create( + MetaStoreUtils.DEFAULT_DATABASE_NAME, "demo_table_3", null)); + job.setOutputFormatClass(HCatOutputFormat.class); + HCatOutputFormat.setSchema(job, getSchema()); + assertTrue(job.waitForCompletion(true)); + if (!HCatUtil.isHadoop23()) { + new FileOutputCommitterContainer(job, null).commitJob(job); + } + assertTrue(job.isSuccessful()); + + server.setBatchOn(); + server.registerQuery("D = load 'default.demo_table_3' using org.apache.hcatalog.pig.HCatLoader();"); + server.executeBatch(); + Iterator XIter = server.openIterator("D"); + int numTuplesRead = 0; + while (XIter.hasNext()) { + Tuple t = XIter.next(); + assertEquals(3, t.size()); + assertEquals(t.get(0).toString(), "" + numTuplesRead); + assertEquals(t.get(1).toString(), "a" + numTuplesRead); + assertEquals(t.get(2).toString(), "b" + numTuplesRead); + numTuplesRead++; + } + assertEquals(input.length, numTuplesRead); + } + + + public static class Map extends Mapper { + + public void map(LongWritable key, Text value, Context context) throws IOException, InterruptedException { + String[] cols = value.toString().split(","); + DefaultHCatRecord record = new DefaultHCatRecord(3); + record.set(0, Integer.parseInt(cols[0])); + record.set(1, cols[1]); + record.set(2, cols[2]); + context.write(NullWritable.get(), record); + } + } + + private HCatSchema getSchema() throws HCatException { + HCatSchema schema = new HCatSchema(new ArrayList()); + schema.append(new HCatFieldSchema("a0", HCatFieldSchema.Type.INT, + "")); + schema.append(new HCatFieldSchema("a1", + HCatFieldSchema.Type.STRING, "")); + schema.append(new HCatFieldSchema("a2", + HCatFieldSchema.Type.STRING, "")); + return schema; + } + +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/rcfile/TestRCFileInputStorageDriver.java.broken hcatalog/core/src/test/java/org/apache/hcatalog/rcfile/TestRCFileInputStorageDriver.java.broken new file mode 100644 index 0000000..a9c69ea --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/rcfile/TestRCFileInputStorageDriver.java.broken @@ -0,0 +1,294 @@ +/** + * 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.hcatalog.rcfile; + +import java.io.IOException; +import java.io.UnsupportedEncodingException; +import java.util.*; + +import junit.framework.Assert; +import junit.framework.TestCase; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.ql.io.RCFile; +import org.apache.hadoop.hive.ql.io.RCFileOutputFormat; +import org.apache.hadoop.hive.serde.Constants; +import org.apache.hadoop.hive.serde2.ColumnProjectionUtils; +import org.apache.hadoop.hive.serde2.columnar.BytesRefArrayWritable; +import org.apache.hadoop.hive.serde2.columnar.BytesRefWritable; +import org.apache.hadoop.io.compress.DefaultCodec; +import org.apache.hadoop.mapreduce.InputFormat; +import org.apache.hadoop.mapreduce.InputSplit; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.JobID; +import org.apache.hadoop.mapreduce.RecordReader; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.TaskAttemptID; +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.common.HCatUtil; +import org.apache.hcatalog.data.DefaultHCatRecord; +import org.apache.hcatalog.data.HCatDataCheckUtil; +import org.apache.hcatalog.data.HCatRecord; +import org.apache.hcatalog.data.schema.HCatSchema; +import org.apache.hcatalog.rcfile.RCFileInputDriver; +import org.apache.hadoop.hive.shims.ShimLoader; + + +public class TestRCFileInputStorageDriver extends TestCase{ + private static final Configuration conf = new Configuration(); + private static final Path dir = new Path(System.getProperty("test.data.dir", ".") + "/mapred"); + private static final Path file = new Path(dir, "test_rcfile"); + private final HadoopShims.HCatHadoopShims shim = ShimLoader.getHadoopShims().getHCatShim(); + + // Generate sample records to compare against + private byte[][][] getRecords() throws UnsupportedEncodingException { + byte[][] record_1 = {"123".getBytes("UTF-8"), "456".getBytes("UTF-8"), + "789".getBytes("UTF-8"), "1000".getBytes("UTF-8"), + "5.3".getBytes("UTF-8"), "hcatalog and hadoop".getBytes("UTF-8"), + new byte[0], "\\N".getBytes("UTF-8")}; + byte[][] record_2 = {"100".getBytes("UTF-8"), "200".getBytes("UTF-8"), + "123".getBytes("UTF-8"), "1000".getBytes("UTF-8"), + "5.3".getBytes("UTF-8"), "hcatalog and hadoop".getBytes("UTF-8"), + new byte[0], "\\N".getBytes("UTF-8")}; + return new byte[][][]{record_1, record_2}; + } + + // Write sample records to file for individual tests + private BytesRefArrayWritable[] initTestEnvironment() throws IOException { + FileSystem fs = FileSystem.getLocal(conf); + fs.delete(file, true); + + byte [][][] records = getRecords(); + RCFileOutputFormat.setColumnNumber(conf, 8); + RCFile.Writer writer = new RCFile.Writer(fs, conf, file, null, new DefaultCodec()); + + BytesRefArrayWritable bytes = writeBytesToFile(records[0], writer); + BytesRefArrayWritable bytes2 = writeBytesToFile(records[1], writer); + + writer.close(); + return new BytesRefArrayWritable[]{bytes,bytes2}; + } + + private BytesRefArrayWritable writeBytesToFile(byte[][] record, RCFile.Writer writer) throws IOException { + BytesRefArrayWritable bytes = new BytesRefArrayWritable(record.length); + for (int i = 0; i < record.length; i++) { + BytesRefWritable cu = new BytesRefWritable(record[i], 0, record[i].length); + bytes.set(i, cu); + } + writer.append(bytes); + return bytes; + } + + public void testConvertValueToTuple() throws IOException,InterruptedException{ + BytesRefArrayWritable[] bytesArr = initTestEnvironment(); + + HCatSchema schema = buildHiveSchema(); + RCFileInputDriver sd = new RCFileInputDriver(); + JobContext jc = shim.createJobContext(conf, new JobID()); + sd.setInputPath(jc, file.toString()); + InputFormat iF = sd.getInputFormat(null); + InputSplit split = iF.getSplits(jc).get(0); + sd.setOriginalSchema(jc, schema); + sd.setOutputSchema(jc, schema); + sd.initialize(jc, getProps()); + + TaskAttemptContext tac = shim.createTaskAttemptContext(conf, new TaskAttemptID()); + RecordReader rr = iF.createRecordReader(split,tac); + rr.initialize(split, tac); + HCatRecord[] tuples = getExpectedRecords(); + for(int j=0; j < 2; j++){ + Assert.assertTrue(rr.nextKeyValue()); + BytesRefArrayWritable w = (BytesRefArrayWritable)rr.getCurrentValue(); + Assert.assertEquals(bytesArr[j], w); + HCatRecord t = sd.convertToHCatRecord(null,w); + Assert.assertEquals(8, t.size()); + Assert.assertTrue(HCatDataCheckUtil.recordsEqual(t,tuples[j])); + } + } + + public void testPruning() throws IOException,InterruptedException{ + BytesRefArrayWritable[] bytesArr = initTestEnvironment(); + + RCFileInputDriver sd = new RCFileInputDriver(); + JobContext jc = shim.createJobContext(conf, new JobID()); + sd.setInputPath(jc, file.toString()); + InputFormat iF = sd.getInputFormat(null); + InputSplit split = iF.getSplits(jc).get(0); + sd.setOriginalSchema(jc, buildHiveSchema()); + sd.setOutputSchema(jc, buildPrunedSchema()); + + sd.initialize(jc, getProps()); + conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR,jc.getConfiguration().get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR)); + TaskAttemptContext tac = shim.createTaskAttemptContext(conf, new TaskAttemptID()); + RecordReader rr = iF.createRecordReader(split,tac); + rr.initialize(split, tac); + HCatRecord[] tuples = getPrunedRecords(); + for(int j=0; j < 2; j++){ + Assert.assertTrue(rr.nextKeyValue()); + BytesRefArrayWritable w = (BytesRefArrayWritable)rr.getCurrentValue(); + Assert.assertFalse(bytesArr[j].equals(w)); + Assert.assertEquals(w.size(), 8); + HCatRecord t = sd.convertToHCatRecord(null,w); + Assert.assertEquals(5, t.size()); + Assert.assertTrue(HCatDataCheckUtil.recordsEqual(t,tuples[j])); + } + assertFalse(rr.nextKeyValue()); + } + + public void testReorderdCols() throws IOException,InterruptedException{ + BytesRefArrayWritable[] bytesArr = initTestEnvironment(); + + RCFileInputDriver sd = new RCFileInputDriver(); + JobContext jc = shim.createJobContext(conf, new JobID()); + sd.setInputPath(jc, file.toString()); + InputFormat iF = sd.getInputFormat(null); + InputSplit split = iF.getSplits(jc).get(0); + sd.setOriginalSchema(jc, buildHiveSchema()); + sd.setOutputSchema(jc, buildReorderedSchema()); + + sd.initialize(jc, getProps()); + Map map = new HashMap(1); + map.put("part1", "first-part"); + sd.setPartitionValues(jc, map); + conf.set(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR,jc.getConfiguration().get(ColumnProjectionUtils.READ_COLUMN_IDS_CONF_STR)); + TaskAttemptContext tac = shim.createTaskAttemptContext(conf, new TaskAttemptID()); + RecordReader rr = iF.createRecordReader(split,tac); + rr.initialize(split, tac); + HCatRecord[] tuples = getReorderedCols(); + for(int j=0; j < 2; j++){ + Assert.assertTrue(rr.nextKeyValue()); + BytesRefArrayWritable w = (BytesRefArrayWritable)rr.getCurrentValue(); + Assert.assertFalse(bytesArr[j].equals(w)); + Assert.assertEquals(w.size(), 8); + HCatRecord t = sd.convertToHCatRecord(null,w); + Assert.assertEquals(7, t.size()); + Assert.assertTrue(HCatDataCheckUtil.recordsEqual(t,tuples[j])); + } + assertFalse(rr.nextKeyValue()); + } + private HCatRecord[] getExpectedRecords(){ + List rec_1 = new ArrayList(8); + Collections.addAll(rec_1, new Byte("123"), + new Short("456"), + new Integer(789), + new Long(1000L), + new Double(5.3D), + new String("hcatalog and hadoop"), + null, + null); + + HCatRecord tup_1 = new DefaultHCatRecord(rec_1); + + List rec_2 = new ArrayList(8); + Collections.addAll(rec_2, new Byte("100"), + new Short("200"), + new Integer(123), + new Long(1000L), + new Double(5.3D), + new String("hcatalog and hadoop"), + null, + null); + HCatRecord tup_2 = new DefaultHCatRecord(rec_2); + + return new HCatRecord[]{tup_1,tup_2}; + } + + private HCatRecord[] getPrunedRecords(){ + List rec_1 = new ArrayList(8); + Collections.addAll(rec_1, new Byte("123"), + new Integer(789), + new Double(5.3D), + new String("hcatalog and hadoop"), + null); + HCatRecord tup_1 = new DefaultHCatRecord(rec_1); + + List rec_2 = new ArrayList(8); + Collections.addAll(rec_2, new Byte("100"), + new Integer(123), + new Double(5.3D), + new String("hcatalog and hadoop"), + null); + HCatRecord tup_2 = new DefaultHCatRecord(rec_2); + + return new HCatRecord[]{tup_1,tup_2}; + } + + private HCatSchema buildHiveSchema() throws HCatException{ + return new HCatSchema(HCatUtil.getHCatFieldSchemaList(new FieldSchema("atinyint", "tinyint", ""), + new FieldSchema("asmallint", "smallint", ""), + new FieldSchema("aint", "int", ""), + new FieldSchema("along", "bigint", ""), + new FieldSchema("adouble", "double", ""), + new FieldSchema("astring", "string", ""), + new FieldSchema("anullint", "int", ""), + new FieldSchema("anullstring", "string", ""))); + } + + private HCatSchema buildPrunedSchema() throws HCatException{ + return new HCatSchema(HCatUtil.getHCatFieldSchemaList(new FieldSchema("atinyint", "tinyint", ""), + new FieldSchema("aint", "int", ""), + new FieldSchema("adouble", "double", ""), + new FieldSchema("astring", "string", ""), + new FieldSchema("anullint", "int", ""))); + } + + private HCatSchema buildReorderedSchema() throws HCatException{ + return new HCatSchema(HCatUtil.getHCatFieldSchemaList(new FieldSchema("aint", "int", ""), + new FieldSchema("part1", "string", ""), + new FieldSchema("adouble", "double", ""), + new FieldSchema("newCol", "tinyint", ""), + new FieldSchema("astring", "string", ""), + new FieldSchema("atinyint", "tinyint", ""), + new FieldSchema("anullint", "int", ""))); + } + + private HCatRecord[] getReorderedCols(){ + List rec_1 = new ArrayList(7); + Collections.addAll(rec_1, new Integer(789), + new String("first-part"), + new Double(5.3D), + null, // new column + new String("hcatalog and hadoop"), + new Byte("123"), + null); + HCatRecord tup_1 = new DefaultHCatRecord(rec_1); + + List rec_2 = new ArrayList(7); + Collections.addAll(rec_2, new Integer(123), + new String("first-part"), + new Double(5.3D), + null, + new String("hcatalog and hadoop"), + new Byte("100"), + null); + HCatRecord tup_2 = new DefaultHCatRecord(rec_2); + + return new HCatRecord[]{tup_1,tup_2}; + + } + private Properties getProps(){ + Properties props = new Properties(); + props.setProperty(Constants.SERIALIZATION_NULL_FORMAT, "\\N"); + props.setProperty(Constants.SERIALIZATION_FORMAT, "9"); + return props; + } +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java hcatalog/core/src/test/java/org/apache/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java new file mode 100644 index 0000000..d5c4a0a --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java @@ -0,0 +1,248 @@ +/** + * 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.hcatalog.rcfile; + +import java.io.IOException; +import java.io.UnsupportedEncodingException; +import java.util.List; +import java.util.Properties; + +import junit.framework.TestCase; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.ql.io.RCFile; +import org.apache.hadoop.hive.ql.io.RCFileOutputFormat; +import org.apache.hadoop.hive.serde.serdeConstants; +import org.apache.hadoop.hive.serde2.columnar.BytesRefArrayWritable; +import org.apache.hadoop.hive.serde2.columnar.BytesRefWritable; +import org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.compress.CompressionCodec; +import org.apache.hadoop.mapreduce.InputSplit; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.RecordReader; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.TaskAttemptID; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * TestRCFile. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.rcfile.TestRCFileMapReduceInputFormat} instead + */ +public class TestRCFileMapReduceInputFormat extends TestCase { + + private static final Logger LOG = LoggerFactory.getLogger(TestRCFileMapReduceInputFormat.class); + + private static Configuration conf = new Configuration(); + + private static ColumnarSerDe serDe; + + private static Path file; + + private static FileSystem fs; + + private static Properties tbl; + + static { + try { + fs = FileSystem.getLocal(conf); + Path dir = new Path(System.getProperty("test.data.dir", ".") + "/mapred"); + file = new Path(dir, "test_rcfile"); + fs.delete(dir, true); + // the SerDe part is from TestLazySimpleSerDe + serDe = new ColumnarSerDe(); + // Create the SerDe + tbl = createProperties(); + serDe.initialize(conf, tbl); + } catch (Exception e) { + } + } + + private static BytesRefArrayWritable patialS = new BytesRefArrayWritable(); + + private static byte[][] bytesArray = null; + + private static BytesRefArrayWritable s = null; + + static { + try { + bytesArray = new byte[][]{"123".getBytes("UTF-8"), + "456".getBytes("UTF-8"), "789".getBytes("UTF-8"), + "1000".getBytes("UTF-8"), "5.3".getBytes("UTF-8"), + "hive and hadoop".getBytes("UTF-8"), new byte[0], + "NULL".getBytes("UTF-8")}; + s = new BytesRefArrayWritable(bytesArray.length); + s.set(0, new BytesRefWritable("123".getBytes("UTF-8"))); + s.set(1, new BytesRefWritable("456".getBytes("UTF-8"))); + s.set(2, new BytesRefWritable("789".getBytes("UTF-8"))); + s.set(3, new BytesRefWritable("1000".getBytes("UTF-8"))); + s.set(4, new BytesRefWritable("5.3".getBytes("UTF-8"))); + s.set(5, new BytesRefWritable("hive and hadoop".getBytes("UTF-8"))); + s.set(6, new BytesRefWritable("NULL".getBytes("UTF-8"))); + s.set(7, new BytesRefWritable("NULL".getBytes("UTF-8"))); + + // partial test init + patialS.set(0, new BytesRefWritable("NULL".getBytes("UTF-8"))); + patialS.set(1, new BytesRefWritable("NULL".getBytes("UTF-8"))); + patialS.set(2, new BytesRefWritable("789".getBytes("UTF-8"))); + patialS.set(3, new BytesRefWritable("1000".getBytes("UTF-8"))); + patialS.set(4, new BytesRefWritable("NULL".getBytes("UTF-8"))); + patialS.set(5, new BytesRefWritable("NULL".getBytes("UTF-8"))); + patialS.set(6, new BytesRefWritable("NULL".getBytes("UTF-8"))); + patialS.set(7, new BytesRefWritable("NULL".getBytes("UTF-8"))); + + } catch (UnsupportedEncodingException e) { + } + } + + + /** For debugging and testing. */ + public static void main(String[] args) throws Exception { + int count = 10000; + boolean create = true; + + String usage = "Usage: RCFile " + "[-count N]" + " file"; + if (args.length == 0) { + LOG.error(usage); + System.exit(-1); + } + + try { + for (int i = 0; i < args.length; ++i) { // parse command line + if (args[i] == null) { + continue; + } else if (args[i].equals("-count")) { + count = Integer.parseInt(args[++i]); + } else { + // file is required parameter + file = new Path(args[i]); + } + } + + if (file == null) { + LOG.error(usage); + System.exit(-1); + } + + LOG.info("count = {}", count); + LOG.info("create = {}", create); + LOG.info("file = {}", file); + + // test.performanceTest(); + LOG.info("Finished."); + } finally { + fs.close(); + } + } + + private static Properties createProperties() { + Properties tbl = new Properties(); + + // Set the configuration parameters + tbl.setProperty(serdeConstants.SERIALIZATION_FORMAT, "9"); + tbl.setProperty("columns", + "abyte,ashort,aint,along,adouble,astring,anullint,anullstring"); + tbl.setProperty("columns.types", + "tinyint:smallint:int:bigint:double:string:int:string"); + tbl.setProperty(serdeConstants.SERIALIZATION_NULL_FORMAT, "NULL"); + return tbl; + } + + + public void testSynAndSplit() throws IOException, InterruptedException { + splitBeforeSync(); + splitRightBeforeSync(); + splitInMiddleOfSync(); + splitRightAfterSync(); + splitAfterSync(); + } + + private void splitBeforeSync() throws IOException, InterruptedException { + writeThenReadByRecordReader(600, 1000, 2, 17684, null); + } + + private void splitRightBeforeSync() throws IOException, InterruptedException { + writeThenReadByRecordReader(500, 1000, 2, 17750, null); + } + + private void splitInMiddleOfSync() throws IOException, InterruptedException { + writeThenReadByRecordReader(500, 1000, 2, 17760, null); + + } + + private void splitRightAfterSync() throws IOException, InterruptedException { + writeThenReadByRecordReader(500, 1000, 2, 17770, null); + } + + private void splitAfterSync() throws IOException, InterruptedException { + writeThenReadByRecordReader(500, 1000, 2, 19950, null); + } + + private void writeThenReadByRecordReader(int intervalRecordCount, + int writeCount, int splitNumber, long maxSplitSize, CompressionCodec codec) + throws IOException, InterruptedException { + Path testDir = new Path(System.getProperty("test.data.dir", ".") + + "/mapred/testsmallfirstsplit"); + Path testFile = new Path(testDir, "test_rcfile"); + fs.delete(testFile, true); + Configuration cloneConf = new Configuration(conf); + RCFileOutputFormat.setColumnNumber(cloneConf, bytesArray.length); + cloneConf.setInt(RCFile.RECORD_INTERVAL_CONF_STR, intervalRecordCount); + + RCFile.Writer writer = new RCFile.Writer(fs, cloneConf, testFile, null, codec); + + BytesRefArrayWritable bytes = new BytesRefArrayWritable(bytesArray.length); + for (int i = 0; i < bytesArray.length; i++) { + BytesRefWritable cu = null; + cu = new BytesRefWritable(bytesArray[i], 0, bytesArray[i].length); + bytes.set(i, cu); + } + for (int i = 0; i < writeCount; i++) { + writer.append(bytes); + } + writer.close(); + + RCFileMapReduceInputFormat inputFormat = new RCFileMapReduceInputFormat(); + Configuration jonconf = new Configuration(cloneConf); + jonconf.set("mapred.input.dir", testDir.toString()); + JobContext context = new Job(jonconf); + context.getConfiguration().setLong("mapred.max.split.size", maxSplitSize); + List splits = inputFormat.getSplits(context); + assertEquals("splits length should be " + splitNumber, splits.size(), splitNumber); + int readCount = 0; + for (int i = 0; i < splits.size(); i++) { + TaskAttemptContext tac = ShimLoader.getHadoopShims().getHCatShim().createTaskAttemptContext(jonconf, new TaskAttemptID()); + RecordReader rr = inputFormat.createRecordReader(splits.get(i), tac); + rr.initialize(splits.get(i), tac); + while (rr.nextKeyValue()) { + readCount++; + } + } + assertEquals("readCount should be equal to writeCount", readCount, writeCount); + } + +} + + diff --git hcatalog/core/src/test/java/org/apache/hcatalog/rcfile/TestRCFileOutputStorageDriver.java.broken hcatalog/core/src/test/java/org/apache/hcatalog/rcfile/TestRCFileOutputStorageDriver.java.broken new file mode 100644 index 0000000..1042501 --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/rcfile/TestRCFileOutputStorageDriver.java.broken @@ -0,0 +1,105 @@ +/** + * 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.hcatalog.rcfile; + +import java.io.IOException; +import java.io.UnsupportedEncodingException; +import java.util.ArrayList; +import java.util.List; +import java.util.Properties; + +import junit.framework.TestCase; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.serde2.columnar.BytesRefArrayWritable; +import org.apache.hadoop.hive.serde2.columnar.BytesRefWritable; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.JobID; +import org.apache.hcatalog.common.HCatConstants; +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.common.HCatUtil; +import org.apache.hcatalog.data.HCatRecord; +import org.apache.hcatalog.data.schema.HCatSchema; +import org.apache.hcatalog.mapreduce.HCatInputStorageDriver; +import org.apache.hcatalog.mapreduce.HCatOutputStorageDriver; +import org.apache.hcatalog.mapreduce.OutputJobInfo; +import org.apache.hadoop.hive.shims.ShimLoader; + +public class TestRCFileOutputStorageDriver extends TestCase { + + public void testConversion() throws IOException { + Configuration conf = new Configuration(); + JobContext jc = ShimLoader.getHadoopShims().getHCatShim().createJobContext(conf, new JobID()); + String jobString = HCatUtil.serialize(OutputJobInfo.create(null,null,null)); + jc.getConfiguration().set(HCatConstants.HCAT_KEY_OUTPUT_INFO,jobString); + + HCatSchema schema = buildHiveSchema(); + HCatInputStorageDriver isd = new RCFileInputDriver(); + + isd.setOriginalSchema(jc, schema); + isd.setOutputSchema(jc, schema); + isd.initialize(jc, new Properties()); + + byte[][] byteArray = buildBytesArray(); + + BytesRefArrayWritable bytesWritable = new BytesRefArrayWritable(byteArray.length); + for (int i = 0; i < byteArray.length; i++) { + BytesRefWritable cu = new BytesRefWritable(byteArray[i], 0, byteArray[i].length); + bytesWritable.set(i, cu); + } + + //Convert byte array to HCatRecord using isd, convert hcatrecord back to byte array + //using osd, compare the two arrays + HCatRecord record = isd.convertToHCatRecord(null, bytesWritable); + + HCatOutputStorageDriver osd = new RCFileOutputDriver(); + + osd.setSchema(jc, schema); + osd.initialize(jc, new Properties()); + + BytesRefArrayWritable bytesWritableOutput = (BytesRefArrayWritable) osd.convertValue(record); + + assertTrue(bytesWritableOutput.compareTo(bytesWritable) == 0); + } + + private byte[][] buildBytesArray() throws UnsupportedEncodingException { + byte[][] bytes = {"123".getBytes("UTF-8"), "456".getBytes("UTF-8"), + "789".getBytes("UTF-8"), "1000".getBytes("UTF-8"), + "5.3".getBytes("UTF-8"), "hcat and hadoop".getBytes("UTF-8"), + new byte[0], "\\N".getBytes("UTF-8") }; + return bytes; + } + + private HCatSchema buildHiveSchema() throws HCatException{ + + List fields = new ArrayList(8); + fields.add(new FieldSchema("atinyint", "tinyint", "")); + fields.add(new FieldSchema("asmallint", "smallint", "")); + fields.add(new FieldSchema("aint", "int", "")); + fields.add(new FieldSchema("along", "bigint", "")); + fields.add(new FieldSchema("adouble", "double", "")); + fields.add(new FieldSchema("astring", "string", "")); + fields.add(new FieldSchema("anullint", "int", "")); + fields.add(new FieldSchema("anullstring", "string", "")); + + return new HCatSchema(HCatUtil.getHCatFieldSchemaList(fields)); + } +} diff --git hcatalog/core/src/test/java/org/apache/hcatalog/security/TestHdfsAuthorizationProvider.java hcatalog/core/src/test/java/org/apache/hcatalog/security/TestHdfsAuthorizationProvider.java new file mode 100644 index 0000000..be5c737 --- /dev/null +++ hcatalog/core/src/test/java/org/apache/hcatalog/security/TestHdfsAuthorizationProvider.java @@ -0,0 +1,586 @@ +/** + * 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.hcatalog.security; + +import static org.apache.hcatalog.HcatTestUtils.perm300; +import static org.apache.hcatalog.HcatTestUtils.perm500; +import static org.apache.hcatalog.HcatTestUtils.perm555; +import static org.apache.hcatalog.HcatTestUtils.perm700; +import static org.apache.hcatalog.HcatTestUtils.perm755; + +import java.io.IOException; +import java.util.Random; + +import junit.framework.Assert; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.cli.CliSessionState; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.Warehouse; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hcatalog.HcatTestUtils; +import org.apache.hcatalog.cli.HCatDriver; +import org.apache.hcatalog.cli.SemanticAnalysis.HCatSemanticAnalyzer; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.security.TestHdfsAuthorizationProvider} instead + */ +public class TestHdfsAuthorizationProvider { + + protected HCatDriver hcatDriver; + protected HiveMetaStoreClient msc; + protected HiveConf conf; + protected String whDir; + protected Path whPath; + protected FileSystem whFs; + protected Warehouse wh; + protected Hive hive; + + @Before + public void setUp() throws Exception { + + conf = new HiveConf(this.getClass()); + conf.set(ConfVars.PREEXECHOOKS.varname, ""); + conf.set(ConfVars.POSTEXECHOOKS.varname, ""); + conf.set(ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); + + conf.set("hive.metastore.local", "true"); + conf.set(ConfVars.SEMANTIC_ANALYZER_HOOK.varname, HCatSemanticAnalyzer.class.getName()); + conf.setBoolVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_ENABLED, true); + conf.setVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER, + StorageDelegationAuthorizationProvider.class.getCanonicalName()); + conf.set("fs.pfile.impl", "org.apache.hadoop.fs.ProxyLocalFileSystem"); + + whDir = System.getProperty("test.warehouse.dir", "/tmp/testhdfsauthorization_wh"); + conf.setVar(HiveConf.ConfVars.METASTOREWAREHOUSE, whDir); + + UserGroupInformation ugi = ShimLoader.getHadoopShims().getUGIForConf(conf); + String username = ShimLoader.getHadoopShims().getShortUserName(ugi); + + whPath = new Path(whDir); + whFs = whPath.getFileSystem(conf); + + wh = new Warehouse(conf); + hive = Hive.get(conf); + + //clean up mess in HMS + HcatTestUtils.cleanupHMS(hive, wh, perm700); + + whFs.delete(whPath, true); + whFs.mkdirs(whPath, perm755); + + SessionState.start(new CliSessionState(conf)); + hcatDriver = new HCatDriver(); + } + + @After + public void tearDown() throws IOException { + whFs.close(); + hcatDriver.close(); + Hive.closeCurrent(); + } + + public Path getDbPath(String dbName) throws MetaException, HiveException { + return HcatTestUtils.getDbPath(hive, wh, dbName); + } + + public Path getTablePath(String dbName, String tableName) throws HiveException { + Table table = hive.getTable(dbName, tableName); + return table.getPath(); + } + + public Path getPartPath(String partName, String dbName, String tableName) throws HiveException { + return new Path(getTablePath(dbName, tableName), partName); + } + + /** Execute the query expecting success*/ + public void exec(String format, Object... args) throws Exception { + String command = String.format(format, args); + CommandProcessorResponse resp = hcatDriver.run(command); + Assert.assertEquals(resp.getErrorMessage(), 0, resp.getResponseCode()); + Assert.assertEquals(resp.getErrorMessage(), null, resp.getErrorMessage()); + } + + /** Execute the query expecting it to fail with AuthorizationException */ + public void execFail(String format, Object... args) throws Exception { + String command = String.format(format, args); + CommandProcessorResponse resp = hcatDriver.run(command); + Assert.assertNotSame(resp.getErrorMessage(), 0, resp.getResponseCode()); + Assert.assertTrue((resp.getResponseCode() == 40000) || (resp.getResponseCode() == 403)); + if (resp.getErrorMessage() != null) { + Assert.assertTrue(resp.getErrorMessage().contains("org.apache.hadoop.security.AccessControlException")); + } + } + + + /** + * Tests whether the warehouse directory is writable by the current user (as defined by Hadoop) + */ + @Test + public void testWarehouseIsWritable() throws Exception { + Path top = new Path(whPath, "_foobarbaz12_"); + try { + whFs.mkdirs(top); + } finally { + whFs.delete(top, true); + } + } + + @Test + public void testShowDatabases() throws Exception { + exec("CREATE DATABASE doo"); + exec("SHOW DATABASES"); + + whFs.setPermission(whPath, perm300); //revoke r + execFail("SHOW DATABASES"); + } + + @Test + public void testDatabaseOps() throws Exception { + exec("SHOW TABLES"); + exec("SHOW TABLE EXTENDED LIKE foo1"); + + whFs.setPermission(whPath, perm700); + exec("CREATE DATABASE doo"); + exec("DESCRIBE DATABASE doo"); + exec("USE doo"); + exec("SHOW TABLES"); + exec("SHOW TABLE EXTENDED LIKE foo1"); + exec("DROP DATABASE doo"); + + //custom location + Path dbPath = new Path(whPath, new Random().nextInt() + "/mydb"); + whFs.mkdirs(dbPath, perm700); + exec("CREATE DATABASE doo2 LOCATION '%s'", dbPath.toUri()); + exec("DESCRIBE DATABASE doo2", dbPath.toUri()); + exec("USE doo2"); + exec("SHOW TABLES"); + exec("SHOW TABLE EXTENDED LIKE foo1"); + exec("DROP DATABASE doo2", dbPath.toUri()); + + //custom non-existing location + exec("CREATE DATABASE doo3 LOCATION '%s/subpath'", dbPath.toUri()); + } + + @Test + public void testCreateDatabaseFail1() throws Exception { + whFs.setPermission(whPath, perm500); + execFail("CREATE DATABASE doo"); //in the default location + + whFs.setPermission(whPath, perm555); + execFail("CREATE DATABASE doo2"); + } + + @Test + public void testCreateDatabaseFail2() throws Exception { + //custom location + Path dbPath = new Path(whPath, new Random().nextInt() + "/mydb"); + + whFs.mkdirs(dbPath, perm700); + whFs.setPermission(dbPath, perm500); + execFail("CREATE DATABASE doo2 LOCATION '%s'", dbPath.toUri()); + } + + @Test + public void testDropDatabaseFail1() throws Exception { + whFs.setPermission(whPath, perm700); + exec("CREATE DATABASE doo"); //in the default location + + whFs.setPermission(getDbPath("doo"), perm500); //revoke write + execFail("DROP DATABASE doo"); + } + + @Test + public void testDropDatabaseFail2() throws Exception { + //custom location + Path dbPath = new Path(whPath, new Random().nextInt() + "/mydb"); + + whFs.mkdirs(dbPath, perm700); + exec("CREATE DATABASE doo2 LOCATION '%s'", dbPath.toUri()); + + whFs.setPermission(dbPath, perm500); + execFail("DROP DATABASE doo2"); + } + + @Test + public void testDescSwitchDatabaseFail() throws Exception { + whFs.setPermission(whPath, perm700); + exec("CREATE DATABASE doo"); + whFs.setPermission(getDbPath("doo"), perm300); //revoke read + execFail("DESCRIBE DATABASE doo"); + execFail("USE doo"); + + //custom location + Path dbPath = new Path(whPath, new Random().nextInt() + "/mydb"); + whFs.mkdirs(dbPath, perm700); + exec("CREATE DATABASE doo2 LOCATION '%s'", dbPath.toUri()); + whFs.mkdirs(dbPath, perm300); //revoke read + execFail("DESCRIBE DATABASE doo2", dbPath.toUri()); + execFail("USE doo2"); + } + + @Test + public void testShowTablesFail() throws Exception { + whFs.setPermission(whPath, perm700); + exec("CREATE DATABASE doo"); + exec("USE doo"); + whFs.setPermission(getDbPath("doo"), perm300); //revoke read + execFail("SHOW TABLES"); + execFail("SHOW TABLE EXTENDED LIKE foo1"); + } + + @Test + public void testTableOps() throws Exception { + //default db + exec("CREATE TABLE foo1 (foo INT) STORED AS RCFILE"); + exec("DESCRIBE foo1"); + exec("DROP TABLE foo1"); + + //default db custom location + Path tablePath = new Path(whPath, new Random().nextInt() + "/mytable"); + whFs.mkdirs(tablePath, perm700); + exec("CREATE EXTERNAL TABLE foo2 (foo INT) STORED AS RCFILE LOCATION '%s'", tablePath); + exec("DESCRIBE foo2"); + exec("DROP TABLE foo2"); + + //default db custom non existing location + exec("CREATE EXTERNAL TABLE foo3 (foo INT) STORED AS RCFILE LOCATION '%s/subpath'", tablePath); + exec("DESCRIBE foo3"); + exec("DROP TABLE foo3"); + + //non default db + exec("CREATE DATABASE doo"); + exec("USE doo"); + + exec("CREATE TABLE foo4 (foo INT) STORED AS RCFILE"); + exec("DESCRIBE foo4"); + exec("DROP TABLE foo4"); + + //non-default db custom location + tablePath = new Path(whPath, new Random().nextInt() + "/mytable"); + whFs.mkdirs(tablePath, perm700); + exec("CREATE EXTERNAL TABLE foo5 (foo INT) STORED AS RCFILE LOCATION '%s'", tablePath); + exec("DESCRIBE foo5"); + exec("DROP TABLE foo5"); + + //non-default db custom non existing location + exec("CREATE EXTERNAL TABLE foo6 (foo INT) STORED AS RCFILE LOCATION '%s/subpath'", tablePath); + exec("DESCRIBE foo6"); + exec("DROP TABLE foo6"); + + exec("DROP TABLE IF EXISTS foo_non_exists"); + + exec("CREATE TABLE foo1 (foo INT) STORED AS RCFILE"); + exec("DESCRIBE EXTENDED foo1"); + exec("DESCRIBE FORMATTED foo1"); + exec("DESCRIBE foo1.foo"); + + //deep non-existing path for the table + tablePath = new Path(whPath, new Random().nextInt() + "/mytable"); + whFs.mkdirs(tablePath, perm700); + exec("CREATE EXTERNAL TABLE foo2 (foo INT) STORED AS RCFILE LOCATION '%s/a/a/a/'", tablePath); + } + + @Test + public void testCreateTableFail1() throws Exception { + //default db + whFs.mkdirs(whPath, perm500); //revoke w + execFail("CREATE TABLE foo1 (foo INT) STORED AS RCFILE"); + } + + @Test + public void testCreateTableFail2() throws Exception { + //default db custom location + Path tablePath = new Path(whPath, new Random().nextInt() + "/mytable"); + whFs.mkdirs(tablePath, perm500); + execFail("CREATE EXTERNAL TABLE foo2 (foo INT) STORED AS RCFILE LOCATION '%s'", tablePath); + + //default db custom non existing location + execFail("CREATE EXTERNAL TABLE foo3 (foo INT) STORED AS RCFILE LOCATION '%s/subpath'", tablePath); + } + + @Test + public void testCreateTableFail3() throws Exception { + //non default db + exec("CREATE DATABASE doo"); + whFs.setPermission(getDbPath("doo"), perm500); + + execFail("CREATE TABLE doo.foo4 (foo INT) STORED AS RCFILE"); + + //non-default db custom location, permission to write to tablePath, but not on db path + Path tablePath = new Path(whPath, new Random().nextInt() + "/mytable"); + whFs.mkdirs(tablePath, perm700); + exec("USE doo"); + execFail("CREATE EXTERNAL TABLE foo5 (foo INT) STORED AS RCFILE LOCATION '%s'", tablePath); + } + + @Test + public void testCreateTableFail4() throws Exception { + //non default db + exec("CREATE DATABASE doo"); + + //non-default db custom location + Path tablePath = new Path(whPath, new Random().nextInt() + "/mytable"); + whFs.mkdirs(tablePath, perm500); + execFail("CREATE EXTERNAL TABLE doo.foo5 (foo INT) STORED AS RCFILE LOCATION '%s'", tablePath); + + //non-default db custom non existing location + execFail("CREATE EXTERNAL TABLE doo.foo6 (foo INT) STORED AS RCFILE LOCATION '%s/a/a/a/'", tablePath); + } + + @Test + public void testDropTableFail1() throws Exception { + //default db + exec("CREATE TABLE foo1 (foo INT) STORED AS RCFILE"); + whFs.mkdirs(getTablePath("default", "foo1"), perm500); //revoke w + execFail("DROP TABLE foo1"); + } + + @Test + public void testDropTableFail2() throws Exception { + //default db custom location + Path tablePath = new Path(whPath, new Random().nextInt() + "/mytable"); + exec("CREATE EXTERNAL TABLE foo2 (foo INT) STORED AS RCFILE LOCATION '%s'", tablePath); + whFs.mkdirs(tablePath, perm500); + execFail("DROP TABLE foo2"); + } + + @Test + public void testDropTableFail4() throws Exception { + //non default db + exec("CREATE DATABASE doo"); + + //non-default db custom location + Path tablePath = new Path(whPath, new Random().nextInt() + "/mytable"); + + exec("CREATE EXTERNAL TABLE doo.foo5 (foo INT) STORED AS RCFILE LOCATION '%s'", tablePath); + whFs.mkdirs(tablePath, perm500); + exec("USE doo"); //There is no DROP TABLE doo.foo5 support in Hive + execFail("DROP TABLE foo5"); + } + + @Test + public void testDescTableFail() throws Exception { + //default db + exec("CREATE TABLE foo1 (foo INT) STORED AS RCFILE"); + whFs.mkdirs(getTablePath("default", "foo1"), perm300); //revoke read + execFail("DESCRIBE foo1"); + + //default db custom location + Path tablePath = new Path(whPath, new Random().nextInt() + "/mytable"); + whFs.mkdirs(tablePath, perm700); + exec("CREATE EXTERNAL TABLE foo2 (foo INT) STORED AS RCFILE LOCATION '%s'", tablePath); + whFs.mkdirs(tablePath, perm300); //revoke read + execFail("DESCRIBE foo2"); + } + + @Test + public void testAlterTableRename() throws Exception { + exec("CREATE TABLE foo1 (foo INT) STORED AS RCFILE"); + exec("ALTER TABLE foo1 RENAME TO foo2"); + + Path tablePath = new Path(whPath, new Random().nextInt() + "/mytable"); + exec("CREATE EXTERNAL TABLE foo3 (foo INT) STORED AS RCFILE LOCATION '%s'", tablePath); + exec("ALTER TABLE foo3 RENAME TO foo4"); + } + + @Test + public void testAlterTableRenameFail() throws Exception { + exec("CREATE TABLE foo1 (foo INT) STORED AS RCFILE"); + whFs.mkdirs(getTablePath("default", "foo1"), perm500); //revoke write + execFail("ALTER TABLE foo1 RENAME TO foo2"); + + Path tablePath = new Path(whPath, new Random().nextInt() + "/mytable"); + exec("CREATE EXTERNAL TABLE foo3 (foo INT) STORED AS RCFILE LOCATION '%s'", tablePath); + whFs.mkdirs(tablePath, perm500); //revoke write + execFail("ALTER TABLE foo3 RENAME TO foo4"); + } + + @Test + public void testAlterTableRelocate() throws Exception { + exec("CREATE TABLE foo1 (foo INT) STORED AS RCFILE"); + Path tablePath = new Path(whPath, new Random().nextInt() + "/mytable"); + exec("ALTER TABLE foo1 SET LOCATION '%s'", tablePath.makeQualified(whFs)); + + tablePath = new Path(whPath, new Random().nextInt() + "/mytable2"); + exec("CREATE EXTERNAL TABLE foo3 (foo INT) STORED AS RCFILE LOCATION '%s'", + tablePath.makeQualified(whFs)); + tablePath = new Path(whPath, new Random().nextInt() + "/mytable2"); + exec("ALTER TABLE foo3 SET LOCATION '%s'", tablePath.makeQualified(whFs)); + } + + @Test + public void testAlterTableRelocateFail() throws Exception { + exec("CREATE TABLE foo1 (foo INT) STORED AS RCFILE"); + Path tablePath = new Path(whPath, new Random().nextInt() + "/mytable"); + whFs.mkdirs(tablePath, perm500); //revoke write + execFail("ALTER TABLE foo1 SET LOCATION '%s'", tablePath.makeQualified(whFs)); + + //dont have access to new table loc + tablePath = new Path(whPath, new Random().nextInt() + "/mytable2"); + exec("CREATE EXTERNAL TABLE foo3 (foo INT) STORED AS RCFILE LOCATION '%s'", + tablePath.makeQualified(whFs)); + tablePath = new Path(whPath, new Random().nextInt() + "/mytable2"); + whFs.mkdirs(tablePath, perm500); //revoke write + execFail("ALTER TABLE foo3 SET LOCATION '%s'", tablePath.makeQualified(whFs)); + + //have access to new table loc, but not old table loc + tablePath = new Path(whPath, new Random().nextInt() + "/mytable3"); + exec("CREATE EXTERNAL TABLE foo4 (foo INT) STORED AS RCFILE LOCATION '%s'", + tablePath.makeQualified(whFs)); + whFs.mkdirs(tablePath, perm500); //revoke write + tablePath = new Path(whPath, new Random().nextInt() + "/mytable3"); + execFail("ALTER TABLE foo4 SET LOCATION '%s'", tablePath.makeQualified(whFs)); + } + + @Test + public void testAlterTable() throws Exception { + exec("CREATE TABLE foo1 (foo INT) PARTITIONED BY (b STRING) STORED AS TEXTFILE"); + exec("ALTER TABLE foo1 SET TBLPROPERTIES ('foo'='bar')"); + exec("ALTER TABLE foo1 SET SERDEPROPERTIES ('foo'='bar')"); + exec("ALTER TABLE foo1 ADD COLUMNS (foo2 INT)"); + } + + @Test + public void testAddDropPartition() throws Exception { + exec("CREATE TABLE foo1 (foo INT) PARTITIONED BY (b STRING) STORED AS TEXTFILE"); + exec("ALTER TABLE foo1 ADD PARTITION (b='2010-10-10')"); + exec("ALTER TABLE foo1 ADD IF NOT EXISTS PARTITION (b='2010-10-10')"); + String relPath = new Random().nextInt() + "/mypart"; + exec("ALTER TABLE foo1 ADD PARTITION (b='2010-10-11') LOCATION '%s'", relPath); + + exec("ALTER TABLE foo1 PARTITION (b='2010-10-10') SET FILEFORMAT RCFILE"); + + exec("ALTER TABLE foo1 PARTITION (b='2010-10-10') SET FILEFORMAT INPUTFORMAT " + + "'org.apache.hadoop.hive.ql.io.RCFileInputFormat' OUTPUTFORMAT " + + "'org.apache.hadoop.hive.ql.io.RCFileOutputFormat' inputdriver " + + "'mydriver' outputdriver 'yourdriver'"); + + exec("ALTER TABLE foo1 DROP PARTITION (b='2010-10-10')"); + exec("ALTER TABLE foo1 DROP PARTITION (b='2010-10-11')"); + } + + @Test + public void testAddPartitionFail1() throws Exception { + exec("CREATE TABLE foo1 (foo INT) PARTITIONED BY (b STRING) STORED AS TEXTFILE"); + whFs.mkdirs(getTablePath("default", "foo1"), perm500); + execFail("ALTER TABLE foo1 ADD PARTITION (b='2010-10-10')"); + } + + @Test + public void testAddPartitionFail2() throws Exception { + exec("CREATE TABLE foo1 (foo INT) PARTITIONED BY (b STRING) STORED AS TEXTFILE"); + String relPath = new Random().nextInt() + "/mypart"; + Path partPath = new Path(getTablePath("default", "foo1"), relPath); + whFs.mkdirs(partPath, perm500); + exec("ALTER TABLE foo1 ADD PARTITION (b='2010-10-10') LOCATION '%s'", partPath); + } + + @Test + public void testDropPartitionFail1() throws Exception { + exec("CREATE TABLE foo1 (foo INT) PARTITIONED BY (b STRING) STORED AS TEXTFILE"); + exec("ALTER TABLE foo1 ADD PARTITION (b='2010-10-10')"); + whFs.mkdirs(getPartPath("b=2010-10-10", "default", "foo1"), perm500); + execFail("ALTER TABLE foo1 DROP PARTITION (b='2010-10-10')"); + } + + @Test + public void testDropPartitionFail2() throws Exception { + exec("CREATE TABLE foo1 (foo INT) PARTITIONED BY (b STRING) STORED AS TEXTFILE"); + String relPath = new Random().nextInt() + "/mypart"; + Path partPath = new Path(getTablePath("default", "foo1"), relPath); + whFs.mkdirs(partPath, perm700); + exec("ALTER TABLE foo1 ADD PARTITION (b='2010-10-10') LOCATION '%s'", partPath); + whFs.mkdirs(partPath, perm500); //revoke write + execFail("ALTER TABLE foo1 DROP PARTITION (b='2010-10-10')"); + } + + @Test + public void testAlterTableFail() throws Exception { + exec("CREATE TABLE foo1 (foo INT) PARTITIONED BY (boo STRING) STORED AS TEXTFILE"); + whFs.mkdirs(getTablePath("default", "foo1"), perm500); //revoke write + execFail("ALTER TABLE foo1 SET TBLPROPERTIES ('foo'='bar')"); + execFail("ALTER TABLE foo1 SET SERDEPROPERTIES ('foo'='bar')"); + execFail("ALTER TABLE foo1 ADD COLUMNS (foo2 INT)"); + } + + @Test + public void testShowTables() throws Exception { + exec("CREATE TABLE foo1 (foo INT) PARTITIONED BY (boo STRING) STORED AS TEXTFILE"); + exec("SHOW PARTITIONS foo1"); + + whFs.mkdirs(getTablePath("default", "foo1"), perm300); //revoke read + execFail("SHOW PARTITIONS foo1"); + } + + @Test + public void testAlterTablePartRename() throws Exception { + exec("CREATE TABLE foo1 (foo INT) PARTITIONED BY (b STRING) STORED AS RCFILE"); + Path loc = new Path(whPath, new Random().nextInt() + "/mypart"); + exec("ALTER TABLE foo1 ADD PARTITION (b='2010-10-16') LOCATION '%s'", loc); + exec("ALTER TABLE foo1 PARTITION (b='2010-10-16') RENAME TO PARTITION (b='2010-10-17')"); + } + + @Test + public void testAlterTablePartRenameFail() throws Exception { + exec("CREATE TABLE foo1 (foo INT) PARTITIONED BY (b STRING) STORED AS RCFILE"); + Path loc = new Path(whPath, new Random().nextInt() + "/mypart"); + exec("ALTER TABLE foo1 ADD PARTITION (b='2010-10-16') LOCATION '%s'", loc); + whFs.setPermission(loc, perm500); //revoke w + execFail("ALTER TABLE foo1 PARTITION (b='2010-10-16') RENAME TO PARTITION (b='2010-10-17')"); + } + + @Test + public void testAlterTablePartRelocate() throws Exception { + exec("CREATE TABLE foo1 (foo INT) PARTITIONED BY (b STRING) STORED AS RCFILE"); + exec("ALTER TABLE foo1 ADD PARTITION (b='2010-10-16')"); + Path partPath = new Path(whPath, new Random().nextInt() + "/mypart"); + exec("ALTER TABLE foo1 PARTITION (b='2010-10-16') SET LOCATION '%s'", partPath.makeQualified(whFs)); + } + + @Test + public void testAlterTablePartRelocateFail() throws Exception { + exec("CREATE TABLE foo1 (foo INT) PARTITIONED BY (b STRING) STORED AS RCFILE"); + + Path oldLoc = new Path(whPath, new Random().nextInt() + "/mypart"); + Path newLoc = new Path(whPath, new Random().nextInt() + "/mypart2"); + + exec("ALTER TABLE foo1 ADD PARTITION (b='2010-10-16') LOCATION '%s'", oldLoc); + whFs.mkdirs(oldLoc, perm500); + execFail("ALTER TABLE foo1 PARTITION (b='2010-10-16') SET LOCATION '%s'", newLoc.makeQualified(whFs)); + whFs.mkdirs(oldLoc, perm700); + whFs.mkdirs(newLoc, perm500); + execFail("ALTER TABLE foo1 PARTITION (b='2010-10-16') SET LOCATION '%s'", newLoc.makeQualified(whFs)); + } + +} diff --git hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileInputStorageDriver.java.broken hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileInputStorageDriver.java.broken index 3942b07..368b3ff 100644 --- hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileInputStorageDriver.java.broken +++ hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileInputStorageDriver.java.broken @@ -50,14 +50,14 @@ import org.apache.hcatalog.data.HCatDataCheckUtil; import org.apache.hcatalog.data.HCatRecord; import org.apache.hcatalog.data.schema.HCatSchema; import org.apache.hcatalog.rcfile.RCFileInputDriver; -import org.apache.hcatalog.shims.HCatHadoopShims; +import org.apache.hadoop.hive.shims.ShimLoader; public class TestRCFileInputStorageDriver extends TestCase{ private static final Configuration conf = new Configuration(); private static final Path dir = new Path(System.getProperty("test.data.dir", ".") + "/mapred"); private static final Path file = new Path(dir, "test_rcfile"); - private final HCatHadoopShims shim = HCatHadoopShims.Instance.get(); + private final HCatHadoopShims shim = ShimLoader.getHadoopShims().getHCatShim(); // Generate sample records to compare against private byte[][][] getRecords() throws UnsupportedEncodingException { diff --git hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileOutputStorageDriver.java.broken hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileOutputStorageDriver.java.broken index 90458cb..1042501 100644 --- hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileOutputStorageDriver.java.broken +++ hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileOutputStorageDriver.java.broken @@ -41,13 +41,13 @@ import org.apache.hcatalog.data.schema.HCatSchema; import org.apache.hcatalog.mapreduce.HCatInputStorageDriver; import org.apache.hcatalog.mapreduce.HCatOutputStorageDriver; import org.apache.hcatalog.mapreduce.OutputJobInfo; -import org.apache.hcatalog.shims.HCatHadoopShims; +import org.apache.hadoop.hive.shims.ShimLoader; public class TestRCFileOutputStorageDriver extends TestCase { public void testConversion() throws IOException { Configuration conf = new Configuration(); - JobContext jc = HCatHadoopShims.Instance.get().createJobContext(conf, new JobID()); + JobContext jc = ShimLoader.getHadoopShims().getHCatShim().createJobContext(conf, new JobID()); String jobString = HCatUtil.serialize(OutputJobInfo.create(null,null,null)); jc.getConfiguration().set(HCatConstants.HCAT_KEY_OUTPUT_INFO,jobString); diff --git hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hcatalog/pig/HCatBaseLoader.java hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hcatalog/pig/HCatBaseLoader.java new file mode 100644 index 0000000..5d2fe4c --- /dev/null +++ hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hcatalog/pig/HCatBaseLoader.java @@ -0,0 +1,154 @@ +/** + * 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.hcatalog.pig; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Properties; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.RecordReader; +import org.apache.hcatalog.data.HCatRecord; +import org.apache.hcatalog.data.schema.HCatSchema; +import org.apache.hcatalog.mapreduce.InputJobInfo; +import org.apache.hcatalog.mapreduce.PartInfo; +import org.apache.pig.LoadFunc; +import org.apache.pig.LoadMetadata; +import org.apache.pig.LoadPushDown; +import org.apache.pig.PigException; +import org.apache.pig.ResourceStatistics; +import org.apache.pig.backend.executionengine.ExecException; +import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigSplit; +import org.apache.pig.data.Tuple; +import org.apache.pig.impl.logicalLayer.FrontendException; +import org.apache.pig.impl.util.UDFContext; + +/** + * Base class for HCatLoader and HCatEximLoader + * @deprecated Use/modify {@link org.apache.hive.hcatalog.pig.HCatBaseLoader} instead + */ + +abstract class HCatBaseLoader extends LoadFunc implements LoadMetadata, LoadPushDown { + + protected static final String PRUNE_PROJECTION_INFO = "prune.projection.info"; + + private RecordReader reader; + protected String signature; + + HCatSchema outputSchema = null; + + + @Override + public Tuple getNext() throws IOException { + try { + HCatRecord hr = (HCatRecord) (reader.nextKeyValue() ? reader.getCurrentValue() : null); + Tuple t = PigHCatUtil.transformToTuple(hr, outputSchema); + // TODO : we were discussing an iter interface, and also a LazyTuple + // change this when plans for that solidifies. + return t; + } catch (ExecException e) { + int errCode = 6018; + String errMsg = "Error while reading input"; + throw new ExecException(errMsg, errCode, + PigException.REMOTE_ENVIRONMENT, e); + } catch (Exception eOther) { + int errCode = 6018; + String errMsg = "Error converting read value to tuple"; + throw new ExecException(errMsg, errCode, + PigException.REMOTE_ENVIRONMENT, eOther); + } + + } + + @Override + public void prepareToRead(RecordReader reader, PigSplit arg1) throws IOException { + this.reader = reader; + } + + @Override + public ResourceStatistics getStatistics(String location, Job job) throws IOException { + // statistics not implemented currently + return null; + } + + @Override + public List getFeatures() { + return Arrays.asList(LoadPushDown.OperatorSet.PROJECTION); + } + + @Override + public RequiredFieldResponse pushProjection(RequiredFieldList requiredFieldsInfo) throws FrontendException { + // Store the required fields information in the UDFContext so that we + // can retrieve it later. + storeInUDFContext(signature, PRUNE_PROJECTION_INFO, requiredFieldsInfo); + + // HCat will always prune columns based on what we ask of it - so the + // response is true + return new RequiredFieldResponse(true); + } + + @Override + public void setUDFContextSignature(String signature) { + this.signature = signature; + } + + + // helper methods + protected void storeInUDFContext(String signature, String key, Object value) { + UDFContext udfContext = UDFContext.getUDFContext(); + Properties props = udfContext.getUDFProperties( + this.getClass(), new String[]{signature}); + props.put(key, value); + } + + /** + * A utility method to get the size of inputs. This is accomplished by summing the + * size of all input paths on supported FileSystems. Locations whose size cannot be + * determined are ignored. Note non-FileSystem and unpartitioned locations will not + * report their input size by default. + */ + protected static long getSizeInBytes(InputJobInfo inputJobInfo) throws IOException { + Configuration conf = new Configuration(); + long sizeInBytes = 0; + + for (PartInfo partInfo : inputJobInfo.getPartitions()) { + try { + Path p = new Path(partInfo.getLocation()); + if (p.getFileSystem(conf).isFile(p)) { + sizeInBytes += p.getFileSystem(conf).getFileStatus(p).getLen(); + } else { + FileStatus[] fileStatuses = p.getFileSystem(conf).listStatus(p); + if (fileStatuses != null) { + for (FileStatus child : fileStatuses) { + sizeInBytes += child.getLen(); + } + } + } + } catch (IOException e) { + // Report size to the extent possible. + } + } + + return sizeInBytes; + } +} diff --git hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hcatalog/pig/HCatBaseStorer.java hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hcatalog/pig/HCatBaseStorer.java new file mode 100644 index 0000000..f291521 --- /dev/null +++ hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hcatalog/pig/HCatBaseStorer.java @@ -0,0 +1,443 @@ +/** + * 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.hcatalog.pig; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +import com.google.common.collect.Lists; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.WritableComparable; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.RecordWriter; +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.data.DefaultHCatRecord; +import org.apache.hcatalog.data.HCatRecord; +import org.apache.hcatalog.data.schema.HCatFieldSchema; +import org.apache.hcatalog.data.schema.HCatFieldSchema.Type; +import org.apache.hcatalog.data.schema.HCatSchema; +import org.apache.pig.ResourceSchema; +import org.apache.pig.ResourceStatistics; +import org.apache.pig.StoreFunc; +import org.apache.pig.StoreMetadata; +import org.apache.pig.backend.BackendException; +import org.apache.pig.data.DataBag; +import org.apache.pig.data.DataByteArray; +import org.apache.pig.data.DataType; +import org.apache.pig.data.Tuple; +import org.apache.pig.impl.logicalLayer.FrontendException; +import org.apache.pig.impl.logicalLayer.schema.Schema; +import org.apache.pig.impl.logicalLayer.schema.Schema.FieldSchema; +import org.apache.pig.impl.util.ObjectSerializer; +import org.apache.pig.impl.util.UDFContext; +import org.apache.pig.impl.util.Utils; + +/** + * Base class for HCatStorer and HCatEximStorer + * + * @deprecated Use/modify {@link org.apache.hive.hcatalog.pig.HCatBaseStorer} instead + */ + +abstract class HCatBaseStorer extends StoreFunc implements StoreMetadata { + + private static final List SUPPORTED_INTEGER_CONVERSIONS = + Lists.newArrayList(Type.TINYINT, Type.SMALLINT, Type.INT); + protected static final String COMPUTED_OUTPUT_SCHEMA = "hcat.output.schema"; + protected final List partitionKeys; + protected final Map partitions; + protected Schema pigSchema; + private RecordWriter, HCatRecord> writer; + protected HCatSchema computedSchema; + protected static final String PIG_SCHEMA = "hcat.pig.store.schema"; + protected String sign; + + public HCatBaseStorer(String partSpecs, String schema) throws Exception { + + partitionKeys = new ArrayList(); + partitions = new HashMap(); + if (partSpecs != null && !partSpecs.trim().isEmpty()) { + String[] partKVPs = partSpecs.split(","); + for (String partKVP : partKVPs) { + String[] partKV = partKVP.split("="); + if (partKV.length == 2) { + String partKey = partKV[0].trim(); + partitionKeys.add(partKey); + partitions.put(partKey, partKV[1].trim()); + } else { + throw new FrontendException("Invalid partition column specification. " + partSpecs, PigHCatUtil.PIG_EXCEPTION_CODE); + } + } + } + + if (schema != null) { + pigSchema = Utils.getSchemaFromString(schema); + } + + } + + @Override + public void checkSchema(ResourceSchema resourceSchema) throws IOException { + + /* Schema provided by user and the schema computed by Pig + * at the time of calling store must match. + */ + Schema runtimeSchema = Schema.getPigSchema(resourceSchema); + if (pigSchema != null) { + if (!Schema.equals(runtimeSchema, pigSchema, false, true)) { + throw new FrontendException("Schema provided in store statement doesn't match with the Schema" + + "returned by Pig run-time. Schema provided in HCatStorer: " + pigSchema.toString() + " Schema received from Pig runtime: " + runtimeSchema.toString(), PigHCatUtil.PIG_EXCEPTION_CODE); + } + } else { + pigSchema = runtimeSchema; + } + UDFContext.getUDFContext().getUDFProperties(this.getClass(), new String[]{sign}).setProperty(PIG_SCHEMA, ObjectSerializer.serialize(pigSchema)); + } + + /** Constructs HCatSchema from pigSchema. Passed tableSchema is the existing + * schema of the table in metastore. + */ + protected HCatSchema convertPigSchemaToHCatSchema(Schema pigSchema, HCatSchema tableSchema) throws FrontendException { + List fieldSchemas = new ArrayList(pigSchema.size()); + for (FieldSchema fSchema : pigSchema.getFields()) { + try { + HCatFieldSchema hcatFieldSchema = getColFromSchema(fSchema.alias, tableSchema); + + fieldSchemas.add(getHCatFSFromPigFS(fSchema, hcatFieldSchema)); + } catch (HCatException he) { + throw new FrontendException(he.getMessage(), PigHCatUtil.PIG_EXCEPTION_CODE, he); + } + } + return new HCatSchema(fieldSchemas); + } + + public static boolean removeTupleFromBag(HCatFieldSchema hcatFieldSchema, FieldSchema bagFieldSchema) throws HCatException { + if (hcatFieldSchema != null && hcatFieldSchema.getArrayElementSchema().get(0).getType() != Type.STRUCT) { + return true; + } + // Column was not found in table schema. Its a new column + List tupSchema = bagFieldSchema.schema.getFields(); + if (hcatFieldSchema == null && tupSchema.size() == 1 && (tupSchema.get(0).schema == null || (tupSchema.get(0).type == DataType.TUPLE && tupSchema.get(0).schema.size() == 1))) { + return true; + } + return false; + } + + + private HCatFieldSchema getHCatFSFromPigFS(FieldSchema fSchema, HCatFieldSchema hcatFieldSchema) throws FrontendException, HCatException { + byte type = fSchema.type; + switch (type) { + + case DataType.CHARARRAY: + case DataType.BIGCHARARRAY: + return new HCatFieldSchema(fSchema.alias, Type.STRING, null); + + case DataType.INTEGER: + if (hcatFieldSchema != null) { + if (!SUPPORTED_INTEGER_CONVERSIONS.contains(hcatFieldSchema.getType())) { + throw new FrontendException("Unsupported type: " + type + " in Pig's schema", + PigHCatUtil.PIG_EXCEPTION_CODE); + } + return new HCatFieldSchema(fSchema.alias, hcatFieldSchema.getType(), null); + } else { + return new HCatFieldSchema(fSchema.alias, Type.INT, null); + } + + case DataType.LONG: + return new HCatFieldSchema(fSchema.alias, Type.BIGINT, null); + + case DataType.FLOAT: + return new HCatFieldSchema(fSchema.alias, Type.FLOAT, null); + + case DataType.DOUBLE: + return new HCatFieldSchema(fSchema.alias, Type.DOUBLE, null); + + case DataType.BYTEARRAY: + return new HCatFieldSchema(fSchema.alias, Type.BINARY, null); + + case DataType.BAG: + Schema bagSchema = fSchema.schema; + List arrFields = new ArrayList(1); + FieldSchema field; + // Find out if we need to throw away the tuple or not. + if (removeTupleFromBag(hcatFieldSchema, fSchema)) { + field = bagSchema.getField(0).schema.getField(0); + } else { + field = bagSchema.getField(0); + } + arrFields.add(getHCatFSFromPigFS(field, hcatFieldSchema == null ? null : hcatFieldSchema.getArrayElementSchema().get(0))); + return new HCatFieldSchema(fSchema.alias, Type.ARRAY, new HCatSchema(arrFields), ""); + + case DataType.TUPLE: + List fieldNames = new ArrayList(); + List hcatFSs = new ArrayList(); + HCatSchema structSubSchema = hcatFieldSchema == null ? null : hcatFieldSchema.getStructSubSchema(); + List fields = fSchema.schema.getFields(); + for (int i = 0; i < fields.size(); i++) { + FieldSchema fieldSchema = fields.get(i); + fieldNames.add(fieldSchema.alias); + hcatFSs.add(getHCatFSFromPigFS(fieldSchema, structSubSchema == null ? null : structSubSchema.get(i))); + } + return new HCatFieldSchema(fSchema.alias, Type.STRUCT, new HCatSchema(hcatFSs), ""); + + case DataType.MAP: { + // Pig's schema contain no type information about map's keys and + // values. So, if its a new column assume if its existing + // return whatever is contained in the existing column. + + HCatFieldSchema valFS; + List valFSList = new ArrayList(1); + + if (hcatFieldSchema != null) { + return new HCatFieldSchema(fSchema.alias, Type.MAP, Type.STRING, hcatFieldSchema.getMapValueSchema(), ""); + } + + // Column not found in target table. Its a new column. Its schema is map + valFS = new HCatFieldSchema(fSchema.alias, Type.STRING, ""); + valFSList.add(valFS); + return new HCatFieldSchema(fSchema.alias, Type.MAP, Type.STRING, new HCatSchema(valFSList), ""); + } + + default: + throw new FrontendException("Unsupported type: " + type + " in Pig's schema", PigHCatUtil.PIG_EXCEPTION_CODE); + } + } + + @Override + public void prepareToWrite(RecordWriter writer) throws IOException { + this.writer = writer; + computedSchema = (HCatSchema) ObjectSerializer.deserialize(UDFContext.getUDFContext().getUDFProperties(this.getClass(), new String[]{sign}).getProperty(COMPUTED_OUTPUT_SCHEMA)); + } + + @Override + public void putNext(Tuple tuple) throws IOException { + + List outgoing = new ArrayList(tuple.size()); + + int i = 0; + for (HCatFieldSchema fSchema : computedSchema.getFields()) { + outgoing.add(getJavaObj(tuple.get(i++), fSchema)); + } + try { + writer.write(null, new DefaultHCatRecord(outgoing)); + } catch (InterruptedException e) { + throw new BackendException("Error while writing tuple: " + tuple, PigHCatUtil.PIG_EXCEPTION_CODE, e); + } + } + + private Object getJavaObj(Object pigObj, HCatFieldSchema hcatFS) throws HCatException, BackendException { + try { + + // The real work-horse. Spend time and energy in this method if there is + // need to keep HCatStorer lean and go fast. + Type type = hcatFS.getType(); + switch (type) { + + case BINARY: + if (pigObj == null) { + return null; + } + return ((DataByteArray) pigObj).get(); + + case STRUCT: + if (pigObj == null) { + return null; + } + HCatSchema structSubSchema = hcatFS.getStructSubSchema(); + // Unwrap the tuple. + List all = ((Tuple) pigObj).getAll(); + ArrayList converted = new ArrayList(all.size()); + for (int i = 0; i < all.size(); i++) { + converted.add(getJavaObj(all.get(i), structSubSchema.get(i))); + } + return converted; + + case ARRAY: + if (pigObj == null) { + return null; + } + // Unwrap the bag. + DataBag pigBag = (DataBag) pigObj; + HCatFieldSchema tupFS = hcatFS.getArrayElementSchema().get(0); + boolean needTuple = tupFS.getType() == Type.STRUCT; + List bagContents = new ArrayList((int) pigBag.size()); + Iterator bagItr = pigBag.iterator(); + + while (bagItr.hasNext()) { + // If there is only one element in tuple contained in bag, we throw away the tuple. + bagContents.add(getJavaObj(needTuple ? bagItr.next() : bagItr.next().get(0), tupFS)); + + } + return bagContents; + case MAP: + if (pigObj == null) { + return null; + } + Map pigMap = (Map) pigObj; + Map typeMap = new HashMap(); + for (Entry entry : pigMap.entrySet()) { + // the value has a schema and not a FieldSchema + typeMap.put( + // Schema validation enforces that the Key is a String + (String) entry.getKey(), + getJavaObj(entry.getValue(), hcatFS.getMapValueSchema().get(0))); + } + return typeMap; + case STRING: + case INT: + case BIGINT: + case FLOAT: + case DOUBLE: + return pigObj; + case SMALLINT: + if (pigObj == null) { + return null; + } + if ((Integer) pigObj < Short.MIN_VALUE || (Integer) pigObj > Short.MAX_VALUE) { + throw new BackendException("Value " + pigObj + " is outside the bounds of column " + + hcatFS.getName() + " with type " + hcatFS.getType(), PigHCatUtil.PIG_EXCEPTION_CODE); + } + return ((Integer) pigObj).shortValue(); + case TINYINT: + if (pigObj == null) { + return null; + } + if ((Integer) pigObj < Byte.MIN_VALUE || (Integer) pigObj > Byte.MAX_VALUE) { + throw new BackendException("Value " + pigObj + " is outside the bounds of column " + + hcatFS.getName() + " with type " + hcatFS.getType(), PigHCatUtil.PIG_EXCEPTION_CODE); + } + return ((Integer) pigObj).byteValue(); + case BOOLEAN: + // would not pass schema validation anyway + throw new BackendException("Incompatible type " + type + " found in hcat table schema: " + hcatFS, PigHCatUtil.PIG_EXCEPTION_CODE); + default: + throw new BackendException("Unexpected type " + type + " for value " + pigObj + (pigObj == null ? "" : " of class " + pigObj.getClass().getName()), PigHCatUtil.PIG_EXCEPTION_CODE); + } + } catch (BackendException e) { + // provide the path to the field in the error message + throw new BackendException( + (hcatFS.getName() == null ? " " : hcatFS.getName() + ".") + e.getMessage(), + e.getCause() == null ? e : e.getCause()); + } + } + + @Override + public String relToAbsPathForStoreLocation(String location, Path curDir) throws IOException { + + // Need to necessarily override this method since default impl assumes HDFS + // based location string. + return location; + } + + @Override + public void setStoreFuncUDFContextSignature(String signature) { + sign = signature; + } + + + protected void doSchemaValidations(Schema pigSchema, HCatSchema tblSchema) throws FrontendException, HCatException { + + // Iterate through all the elements in Pig Schema and do validations as + // dictated by semantics, consult HCatSchema of table when need be. + + for (FieldSchema pigField : pigSchema.getFields()) { + HCatFieldSchema hcatField = getColFromSchema(pigField.alias, tblSchema); + validateSchema(pigField, hcatField); + } + + try { + PigHCatUtil.validateHCatTableSchemaFollowsPigRules(tblSchema); + } catch (IOException e) { + throw new FrontendException("HCatalog schema is not compatible with Pig: " + e.getMessage(), PigHCatUtil.PIG_EXCEPTION_CODE, e); + } + } + + + private void validateSchema(FieldSchema pigField, HCatFieldSchema hcatField) + throws HCatException, FrontendException { + validateAlias(pigField.alias); + byte type = pigField.type; + if (DataType.isComplex(type)) { + switch (type) { + + case DataType.MAP: + if (hcatField != null) { + if (hcatField.getMapKeyType() != Type.STRING) { + throw new FrontendException("Key Type of map must be String " + hcatField, PigHCatUtil.PIG_EXCEPTION_CODE); + } + // Map values can be primitive or complex + } + break; + + case DataType.BAG: + HCatSchema arrayElementSchema = hcatField == null ? null : hcatField.getArrayElementSchema(); + for (FieldSchema innerField : pigField.schema.getField(0).schema.getFields()) { + validateSchema(innerField, getColFromSchema(pigField.alias, arrayElementSchema)); + } + break; + + case DataType.TUPLE: + HCatSchema structSubSchema = hcatField == null ? null : hcatField.getStructSubSchema(); + for (FieldSchema innerField : pigField.schema.getFields()) { + validateSchema(innerField, getColFromSchema(pigField.alias, structSubSchema)); + } + break; + + default: + throw new FrontendException("Internal Error.", PigHCatUtil.PIG_EXCEPTION_CODE); + } + } + } + + private void validateAlias(String alias) throws FrontendException { + if (alias == null) { + throw new FrontendException("Column name for a field is not specified. Please provide the full schema as an argument to HCatStorer.", PigHCatUtil.PIG_EXCEPTION_CODE); + } + if (alias.matches(".*[A-Z]+.*")) { + throw new FrontendException("Column names should all be in lowercase. Invalid name found: " + alias, PigHCatUtil.PIG_EXCEPTION_CODE); + } + } + + // Finds column by name in HCatSchema, if not found returns null. + private HCatFieldSchema getColFromSchema(String alias, HCatSchema tblSchema) { + if (tblSchema != null) { + for (HCatFieldSchema hcatField : tblSchema.getFields()) { + if (hcatField != null && hcatField.getName() != null && hcatField.getName().equalsIgnoreCase(alias)) { + return hcatField; + } + } + } + // Its a new column + return null; + } + + @Override + public void cleanupOnFailure(String location, Job job) throws IOException { + // No-op. + } + + @Override + public void storeStatistics(ResourceStatistics stats, String arg1, Job job) throws IOException { + } +} diff --git hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hcatalog/pig/HCatEximLoader.java.broken hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hcatalog/pig/HCatEximLoader.java.broken new file mode 100644 index 0000000..33824b8 --- /dev/null +++ hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hcatalog/pig/HCatEximLoader.java.broken @@ -0,0 +1,129 @@ +/** + * 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.hcatalog.pig; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Properties; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hcatalog.data.schema.HCatFieldSchema; +import org.apache.hcatalog.data.schema.HCatSchema; +import org.apache.hcatalog.mapreduce.HCatBaseInputFormat; +import org.apache.hcatalog.mapreduce.HCatEximInputFormat; +import org.apache.hadoop.mapreduce.InputFormat; +import org.apache.hadoop.mapreduce.Job; +import org.apache.pig.Expression; +import org.apache.pig.LoadFunc; +import org.apache.pig.ResourceSchema; +import org.apache.pig.impl.util.UDFContext; + +/** + * Pig {@link LoadFunc} to read data/metadata from hcatalog exported location + */ + +public class HCatEximLoader extends HCatBaseLoader { + + private static final Log LOG = LogFactory.getLog(HCatEximLoader.class); + + private HCatSchema tableSchema; + private HCatSchema partitionSchema; + private HCatEximInputFormat inputFormat; + + public HCatEximLoader() { + LOG.debug("HCatEximLoader ctored"); + } + + @Override + public ResourceSchema getSchema(String location, Job job) throws IOException { + LOG.debug("getSchema with location :" + location); + if (tableSchema == null) { + List rv = HCatEximInputFormat.setInput(job, location, null); + tableSchema = rv.get(0); + partitionSchema = rv.get(1); + } + LOG.debug("getSchema got schema :" + tableSchema.toString()); + List colsPlusPartKeys = new ArrayList(); + colsPlusPartKeys.addAll(tableSchema.getFields()); + colsPlusPartKeys.addAll(partitionSchema.getFields()); + outputSchema = new HCatSchema(colsPlusPartKeys); + return PigHCatUtil.getResourceSchema(outputSchema); + } + + @Override + public String[] getPartitionKeys(String location, Job job) throws IOException { + LOG.warn("getPartitionKeys with location :" + location); + /* + if (tableSchema == null) { + List rv = HCatEximInputFormat.setInput(job, location, null); + tableSchema = rv.get(0); + partitionSchema = rv.get(1); + } + return partitionSchema.getFieldNames().toArray(new String[0]); + */ + return null; + } + + @Override + public void setPartitionFilter(Expression partitionFilter) throws IOException { + LOG.debug("setPartitionFilter with filter :" + partitionFilter.toString()); + } + + @Override + public void setLocation(String location, Job job) throws IOException { + LOG.debug("setLocation with location :" + location); + List rv = HCatEximInputFormat.setInput(job, location, null); + tableSchema = rv.get(0); + partitionSchema = rv.get(1); + List colsPlusPartKeys = new ArrayList(); + colsPlusPartKeys.addAll(tableSchema.getFields()); + colsPlusPartKeys.addAll(partitionSchema.getFields()); + outputSchema = new HCatSchema(colsPlusPartKeys); + UDFContext udfContext = UDFContext.getUDFContext(); + Properties props = udfContext.getUDFProperties(this.getClass(), + new String[] {signature}); + RequiredFieldList requiredFieldsInfo = + (RequiredFieldList) props.get(PRUNE_PROJECTION_INFO); + if (requiredFieldsInfo != null) { + ArrayList fcols = new ArrayList(); + for (RequiredField rf : requiredFieldsInfo.getFields()) { + fcols.add(tableSchema.getFields().get(rf.getIndex())); + } + outputSchema = new HCatSchema(fcols); + try { + HCatBaseInputFormat.setOutputSchema(job, outputSchema); + } catch (Exception e) { + throw new IOException(e); + } + } + } + + + @Override + public InputFormat getInputFormat() throws IOException { + if (inputFormat == null) { + inputFormat = new HCatEximInputFormat(); + } + return inputFormat; + } + +} diff --git hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hcatalog/pig/HCatEximStorer.java.broken hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hcatalog/pig/HCatEximStorer.java.broken new file mode 100644 index 0000000..3e21cc7 --- /dev/null +++ hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hcatalog/pig/HCatEximStorer.java.broken @@ -0,0 +1,152 @@ +/** + * 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.hcatalog.pig; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Properties; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hcatalog.common.HCatConstants; +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.common.HCatUtil; +import org.apache.hcatalog.data.schema.HCatFieldSchema; +import org.apache.hcatalog.data.schema.HCatSchema; +import org.apache.hcatalog.mapreduce.HCatEximOutputCommitter; +import org.apache.hcatalog.mapreduce.HCatEximOutputFormat; +import org.apache.hadoop.hive.metastore.MetaStoreUtils; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.OutputFormat; +import org.apache.pig.ResourceSchema; +import org.apache.pig.impl.logicalLayer.FrontendException; +import org.apache.pig.impl.logicalLayer.schema.Schema; +import org.apache.pig.impl.util.ObjectSerializer; +import org.apache.pig.impl.util.UDFContext; + +/** + * HCatEximStorer. + * + */ + +public class HCatEximStorer extends HCatBaseStorer { + + private static final Log LOG = LogFactory.getLog(HCatEximStorer.class); + + private final String outputLocation; + + public HCatEximStorer(String outputLocation) throws Exception { + this(outputLocation, null, null); + } + + public HCatEximStorer(String outputLocation, String partitionSpec) throws Exception { + this(outputLocation, partitionSpec, null); + } + + public HCatEximStorer(String outputLocation, String partitionSpec, String schema) + throws Exception { + super(partitionSpec, schema); + this.outputLocation = outputLocation; + LOG.debug("HCatEximStorer called"); + } + + @Override + public OutputFormat getOutputFormat() throws IOException { + LOG.debug("getOutputFormat called"); + return new HCatEximOutputFormat(); + } + + @Override + public void setStoreLocation(String location, Job job) throws IOException { + LOG.debug("setStoreLocation called with :" + location); + String[] userStr = location.split("\\."); + String dbname = MetaStoreUtils.DEFAULT_DATABASE_NAME; + String tablename = null; + if (userStr.length == 2) { + dbname = userStr[0]; + tablename = userStr[1]; + } else { + tablename = userStr[0]; + } + Properties p = UDFContext.getUDFContext() + .getUDFProperties(this.getClass(), new String[] {sign}); + Configuration config = job.getConfiguration(); + if (!HCatUtil.checkJobContextIfRunningFromBackend(job)) { + Schema schema = (Schema) ObjectSerializer.deserialize(p.getProperty(PIG_SCHEMA)); + if (schema != null) { + pigSchema = schema; + } + if (pigSchema == null) { + throw new FrontendException("Schema for data cannot be determined.", + PigHCatUtil.PIG_EXCEPTION_CODE); + } + HCatSchema hcatTblSchema = new HCatSchema(new ArrayList()); + try { + doSchemaValidations(pigSchema, hcatTblSchema); + } catch (HCatException he) { + throw new FrontendException(he.getMessage(), PigHCatUtil.PIG_EXCEPTION_CODE, he); + } + + List hcatFields = new ArrayList(); + List partVals = new ArrayList(); + for (String key : partitionKeys) { + hcatFields.add(new HCatFieldSchema(key, HCatFieldSchema.Type.STRING, "")); + partVals.add(partitions.get(key)); + } + + HCatSchema outputSchema = convertPigSchemaToHCatSchema(pigSchema, + hcatTblSchema); + LOG.debug("Pig Schema '" + pigSchema.toString() + "' was converted to HCatSchema '" + + outputSchema); + HCatEximOutputFormat.setOutput(job, + dbname, tablename, + outputLocation, + new HCatSchema(hcatFields), + partVals, + outputSchema); + p.setProperty(COMPUTED_OUTPUT_SCHEMA, ObjectSerializer.serialize(outputSchema)); + p.setProperty(HCatConstants.HCAT_KEY_OUTPUT_INFO, + config.get(HCatConstants.HCAT_KEY_OUTPUT_INFO)); + if (config.get(HCatConstants.HCAT_KEY_HIVE_CONF) != null) { + p.setProperty(HCatConstants.HCAT_KEY_HIVE_CONF, + config.get(HCatConstants.HCAT_KEY_HIVE_CONF)); + } + } else { + config.set(HCatConstants.HCAT_KEY_OUTPUT_INFO, + p.getProperty(HCatConstants.HCAT_KEY_OUTPUT_INFO)); + if (p.getProperty(HCatConstants.HCAT_KEY_HIVE_CONF) != null) { + config.set(HCatConstants.HCAT_KEY_HIVE_CONF, + p.getProperty(HCatConstants.HCAT_KEY_HIVE_CONF)); + } + } + } + + @Override + public void storeSchema(ResourceSchema schema, String arg1, Job job) throws IOException { + if( job.getConfiguration().get("mapred.job.tracker", "").equalsIgnoreCase("local") ) { + //In local mode, mapreduce will not call OutputCommitter.cleanupJob. + //Calling it from here so that the partition publish happens. + //This call needs to be removed after MAPREDUCE-1447 is fixed. + new HCatEximOutputCommitter(job,null).cleanupJob(job); + } + } +} diff --git hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hcatalog/pig/HCatLoader.java hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hcatalog/pig/HCatLoader.java new file mode 100644 index 0000000..5d56c61 --- /dev/null +++ hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hcatalog/pig/HCatLoader.java @@ -0,0 +1,274 @@ +/** + * 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.hcatalog.pig; + +import java.io.IOException; +import java.util.Enumeration; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Properties; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.mapreduce.InputFormat; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.security.Credentials; +import org.apache.hcatalog.common.HCatConstants; +import org.apache.hcatalog.common.HCatContext; +import org.apache.hcatalog.common.HCatUtil; +import org.apache.hcatalog.data.Pair; +import org.apache.hcatalog.data.schema.HCatSchema; +import org.apache.hcatalog.mapreduce.HCatInputFormat; +import org.apache.hcatalog.mapreduce.InputJobInfo; +import org.apache.pig.Expression; +import org.apache.pig.Expression.BinaryExpression; +import org.apache.pig.PigException; +import org.apache.pig.ResourceSchema; +import org.apache.pig.ResourceStatistics; +import org.apache.pig.impl.util.UDFContext; + +/** + * Pig {@link org.apache.pig.LoadFunc} to read data from HCat + * @deprecated Use/modify {@link org.apache.hive.hcatalog.pig.HCatLoader} instead + */ + +public class HCatLoader extends HCatBaseLoader { + + private static final String PARTITION_FILTER = "partition.filter"; // for future use + + private HCatInputFormat hcatInputFormat = null; + private String dbName; + private String tableName; + private String hcatServerUri; + private String partitionFilterString; + private final PigHCatUtil phutil = new PigHCatUtil(); + + // Signature for wrapped loader, see comments in LoadFuncBasedInputDriver.initialize + final public static String INNER_SIGNATURE = "hcatloader.inner.signature"; + final public static String INNER_SIGNATURE_PREFIX = "hcatloader_inner_signature"; + // A hash map which stores job credentials. The key is a signature passed by Pig, which is + //unique to the load func and input file name (table, in our case). + private static Map jobCredentials = new HashMap(); + + @Override + public InputFormat getInputFormat() throws IOException { + if (hcatInputFormat == null) { + hcatInputFormat = new HCatInputFormat(); + } + return hcatInputFormat; + } + + @Override + public String relativeToAbsolutePath(String location, Path curDir) throws IOException { + return location; + } + + @Override + public void setLocation(String location, Job job) throws IOException { + HCatContext.INSTANCE.setConf(job.getConfiguration()).getConf().get() + .setBoolean(HCatConstants.HCAT_DATA_TINY_SMALL_INT_PROMOTION, true); + + UDFContext udfContext = UDFContext.getUDFContext(); + Properties udfProps = udfContext.getUDFProperties(this.getClass(), + new String[]{signature}); + job.getConfiguration().set(INNER_SIGNATURE, INNER_SIGNATURE_PREFIX + "_" + signature); + Pair dbTablePair = PigHCatUtil.getDBTableNames(location); + dbName = dbTablePair.first; + tableName = dbTablePair.second; + + RequiredFieldList requiredFieldsInfo = (RequiredFieldList) udfProps + .get(PRUNE_PROJECTION_INFO); + // get partitionFilterString stored in the UDFContext - it would have + // been stored there by an earlier call to setPartitionFilter + // call setInput on HCatInputFormat only in the frontend because internally + // it makes calls to the hcat server - we don't want these to happen in + // the backend + // in the hadoop front end mapred.task.id property will not be set in + // the Configuration + if (udfProps.containsKey(HCatConstants.HCAT_PIG_LOADER_LOCATION_SET)) { + for (Enumeration emr = udfProps.keys(); emr.hasMoreElements(); ) { + PigHCatUtil.getConfigFromUDFProperties(udfProps, + job.getConfiguration(), emr.nextElement().toString()); + } + if (!HCatUtil.checkJobContextIfRunningFromBackend(job)) { + //Combine credentials and credentials from job takes precedence for freshness + Credentials crd = jobCredentials.get(INNER_SIGNATURE_PREFIX + "_" + signature); + crd.addAll(job.getCredentials()); + job.getCredentials().addAll(crd); + } + } else { + Job clone = new Job(job.getConfiguration()); + HCatInputFormat.setInput(job, dbName, tableName).setFilter(getPartitionFilterString()); + + // We will store all the new /changed properties in the job in the + // udf context, so the the HCatInputFormat.setInput method need not + //be called many times. + for (Entry keyValue : job.getConfiguration()) { + String oldValue = clone.getConfiguration().getRaw(keyValue.getKey()); + if ((oldValue == null) || (keyValue.getValue().equals(oldValue) == false)) { + udfProps.put(keyValue.getKey(), keyValue.getValue()); + } + } + udfProps.put(HCatConstants.HCAT_PIG_LOADER_LOCATION_SET, true); + + //Store credentials in a private hash map and not the udf context to + // make sure they are not public. + Credentials crd = new Credentials(); + crd.addAll(job.getCredentials()); + jobCredentials.put(INNER_SIGNATURE_PREFIX + "_" + signature, crd); + } + + // Need to also push projections by calling setOutputSchema on + // HCatInputFormat - we have to get the RequiredFields information + // from the UdfContext, translate it to an Schema and then pass it + // The reason we do this here is because setLocation() is called by + // Pig runtime at InputFormat.getSplits() and + // InputFormat.createRecordReader() time - we are not sure when + // HCatInputFormat needs to know about pruned projections - so doing it + // here will ensure we communicate to HCatInputFormat about pruned + // projections at getSplits() and createRecordReader() time + + if (requiredFieldsInfo != null) { + // convert to hcatschema and pass to HCatInputFormat + try { + outputSchema = phutil.getHCatSchema(requiredFieldsInfo.getFields(), signature, this.getClass()); + HCatInputFormat.setOutputSchema(job, outputSchema); + } catch (Exception e) { + throw new IOException(e); + } + } else { + // else - this means pig's optimizer never invoked the pushProjection + // method - so we need all fields and hence we should not call the + // setOutputSchema on HCatInputFormat + if (HCatUtil.checkJobContextIfRunningFromBackend(job)) { + try { + HCatSchema hcatTableSchema = (HCatSchema) udfProps.get(HCatConstants.HCAT_TABLE_SCHEMA); + outputSchema = hcatTableSchema; + HCatInputFormat.setOutputSchema(job, outputSchema); + } catch (Exception e) { + throw new IOException(e); + } + } + } + + } + + @Override + public String[] getPartitionKeys(String location, Job job) + throws IOException { + Table table = phutil.getTable(location, + hcatServerUri != null ? hcatServerUri : PigHCatUtil.getHCatServerUri(job), + PigHCatUtil.getHCatServerPrincipal(job)); + List tablePartitionKeys = table.getPartitionKeys(); + String[] partitionKeys = new String[tablePartitionKeys.size()]; + for (int i = 0; i < tablePartitionKeys.size(); i++) { + partitionKeys[i] = tablePartitionKeys.get(i).getName(); + } + return partitionKeys; + } + + @Override + public ResourceSchema getSchema(String location, Job job) throws IOException { + HCatContext.INSTANCE.setConf(job.getConfiguration()).getConf().get() + .setBoolean(HCatConstants.HCAT_DATA_TINY_SMALL_INT_PROMOTION, true); + + Table table = phutil.getTable(location, + hcatServerUri != null ? hcatServerUri : PigHCatUtil.getHCatServerUri(job), + PigHCatUtil.getHCatServerPrincipal(job)); + HCatSchema hcatTableSchema = HCatUtil.getTableSchemaWithPtnCols(table); + try { + PigHCatUtil.validateHCatTableSchemaFollowsPigRules(hcatTableSchema); + } catch (IOException e) { + throw new PigException( + "Table schema incompatible for reading through HCatLoader :" + e.getMessage() + + ";[Table schema was " + hcatTableSchema.toString() + "]" + , PigHCatUtil.PIG_EXCEPTION_CODE, e); + } + storeInUDFContext(signature, HCatConstants.HCAT_TABLE_SCHEMA, hcatTableSchema); + outputSchema = hcatTableSchema; + return PigHCatUtil.getResourceSchema(hcatTableSchema); + } + + @Override + public void setPartitionFilter(Expression partitionFilter) throws IOException { + // convert the partition filter expression into a string expected by + // hcat and pass it in setLocation() + + partitionFilterString = getHCatComparisonString(partitionFilter); + + // store this in the udf context so we can get it later + storeInUDFContext(signature, + PARTITION_FILTER, partitionFilterString); + } + + /** + * Get statistics about the data to be loaded. Only input data size is implemented at this time. + */ + @Override + public ResourceStatistics getStatistics(String location, Job job) throws IOException { + try { + ResourceStatistics stats = new ResourceStatistics(); + InputJobInfo inputJobInfo = (InputJobInfo) HCatUtil.deserialize( + job.getConfiguration().get(HCatConstants.HCAT_KEY_JOB_INFO)); + stats.setmBytes(getSizeInBytes(inputJobInfo) / 1024 / 1024); + return stats; + } catch (Exception e) { + throw new IOException(e); + } + } + + private String getPartitionFilterString() { + if (partitionFilterString == null) { + Properties props = UDFContext.getUDFContext().getUDFProperties( + this.getClass(), new String[]{signature}); + partitionFilterString = props.getProperty(PARTITION_FILTER); + } + return partitionFilterString; + } + + private String getHCatComparisonString(Expression expr) { + if (expr instanceof BinaryExpression) { + // call getHCatComparisonString on lhs and rhs, and and join the + // results with OpType string + + // we can just use OpType.toString() on all Expression types except + // Equal, NotEqualt since Equal has '==' in toString() and + // we need '=' + String opStr = null; + switch (expr.getOpType()) { + case OP_EQ: + opStr = " = "; + break; + default: + opStr = expr.getOpType().toString(); + } + BinaryExpression be = (BinaryExpression) expr; + return "(" + getHCatComparisonString(be.getLhs()) + + opStr + + getHCatComparisonString(be.getRhs()) + ")"; + } else { + // should be a constant or column + return expr.toString(); + } + } + +} diff --git hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hcatalog/pig/HCatStorer.java hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hcatalog/pig/HCatStorer.java new file mode 100644 index 0000000..6cbe926 --- /dev/null +++ hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hcatalog/pig/HCatStorer.java @@ -0,0 +1,168 @@ +/** + * 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.hcatalog.pig; + +import java.io.IOException; +import java.util.Enumeration; +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Properties; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.OutputFormat; +import org.apache.hadoop.security.Credentials; +import org.apache.hcatalog.common.HCatConstants; +import org.apache.hcatalog.common.HCatContext; +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.data.schema.HCatSchema; +import org.apache.hcatalog.mapreduce.HCatOutputFormat; +import org.apache.hcatalog.mapreduce.OutputJobInfo; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.pig.PigException; +import org.apache.pig.ResourceSchema; +import org.apache.pig.impl.logicalLayer.FrontendException; +import org.apache.pig.impl.logicalLayer.schema.Schema; +import org.apache.pig.impl.util.ObjectSerializer; +import org.apache.pig.impl.util.UDFContext; + +/** + * HCatStorer. + * + * @deprecated Use/modify {@link org.apache.hive.hcatalog.pig.HCatStorer} instead + */ + +public class HCatStorer extends HCatBaseStorer { + + // Signature for wrapped storer, see comments in LoadFuncBasedInputDriver.initialize + final public static String INNER_SIGNATURE = "hcatstorer.inner.signature"; + final public static String INNER_SIGNATURE_PREFIX = "hcatstorer_inner_signature"; + // A hash map which stores job credentials. The key is a signature passed by Pig, which is + //unique to the store func and out file name (table, in our case). + private static Map jobCredentials = new HashMap(); + + + public HCatStorer(String partSpecs, String schema) throws Exception { + super(partSpecs, schema); + } + + public HCatStorer(String partSpecs) throws Exception { + this(partSpecs, null); + } + + public HCatStorer() throws Exception { + this(null, null); + } + + @Override + public OutputFormat getOutputFormat() throws IOException { + return new HCatOutputFormat(); + } + + @Override + public void setStoreLocation(String location, Job job) throws IOException { + HCatContext.INSTANCE.setConf(job.getConfiguration()).getConf().get() + .setBoolean(HCatConstants.HCAT_DATA_TINY_SMALL_INT_PROMOTION, false); + + Configuration config = job.getConfiguration(); + config.set(INNER_SIGNATURE, INNER_SIGNATURE_PREFIX + "_" + sign); + Properties udfProps = UDFContext.getUDFContext().getUDFProperties( + this.getClass(), new String[]{sign}); + String[] userStr = location.split("\\."); + + if (udfProps.containsKey(HCatConstants.HCAT_PIG_STORER_LOCATION_SET)) { + for (Enumeration emr = udfProps.keys(); emr.hasMoreElements(); ) { + PigHCatUtil.getConfigFromUDFProperties(udfProps, config, emr.nextElement().toString()); + } + Credentials crd = jobCredentials.get(INNER_SIGNATURE_PREFIX + "_" + sign); + if (crd != null) { + job.getCredentials().addAll(crd); + } + } else { + Job clone = new Job(job.getConfiguration()); + OutputJobInfo outputJobInfo; + if (userStr.length == 2) { + outputJobInfo = OutputJobInfo.create(userStr[0], userStr[1], partitions); + } else if (userStr.length == 1) { + outputJobInfo = OutputJobInfo.create(null, userStr[0], partitions); + } else { + throw new FrontendException("location " + location + + " is invalid. It must be of the form [db.]table", + PigHCatUtil.PIG_EXCEPTION_CODE); + } + Schema schema = (Schema) ObjectSerializer.deserialize(udfProps.getProperty(PIG_SCHEMA)); + if (schema != null) { + pigSchema = schema; + } + if (pigSchema == null) { + throw new FrontendException( + "Schema for data cannot be determined.", + PigHCatUtil.PIG_EXCEPTION_CODE); + } + String externalLocation = (String) udfProps.getProperty(HCatConstants.HCAT_PIG_STORER_EXTERNAL_LOCATION); + if (externalLocation != null) { + outputJobInfo.setLocation(externalLocation); + } + try { + HCatOutputFormat.setOutput(job, outputJobInfo); + } catch (HCatException he) { + // pass the message to the user - essentially something about + // the table + // information passed to HCatOutputFormat was not right + throw new PigException(he.getMessage(), + PigHCatUtil.PIG_EXCEPTION_CODE, he); + } + HCatSchema hcatTblSchema = HCatOutputFormat.getTableSchema(job); + try { + doSchemaValidations(pigSchema, hcatTblSchema); + } catch (HCatException he) { + throw new FrontendException(he.getMessage(), PigHCatUtil.PIG_EXCEPTION_CODE, he); + } + computedSchema = convertPigSchemaToHCatSchema(pigSchema, hcatTblSchema); + HCatOutputFormat.setSchema(job, computedSchema); + udfProps.setProperty(COMPUTED_OUTPUT_SCHEMA, ObjectSerializer.serialize(computedSchema)); + + // We will store all the new /changed properties in the job in the + // udf context, so the the HCatOutputFormat.setOutput and setSchema + // methods need not be called many times. + for (Entry keyValue : job.getConfiguration()) { + String oldValue = clone.getConfiguration().getRaw(keyValue.getKey()); + if ((oldValue == null) || (keyValue.getValue().equals(oldValue) == false)) { + udfProps.put(keyValue.getKey(), keyValue.getValue()); + } + } + //Store credentials in a private hash map and not the udf context to + // make sure they are not public. + jobCredentials.put(INNER_SIGNATURE_PREFIX + "_" + sign, job.getCredentials()); + udfProps.put(HCatConstants.HCAT_PIG_STORER_LOCATION_SET, true); + } + } + + @Override + public void storeSchema(ResourceSchema schema, String arg1, Job job) throws IOException { + ShimLoader.getHadoopShims().getHCatShim().commitJob(getOutputFormat(), job); + } + + @Override + public void cleanupOnFailure(String location, Job job) throws IOException { + ShimLoader.getHadoopShims().getHCatShim().abortJob(getOutputFormat(), job); + } +} diff --git hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hcatalog/pig/PigHCatUtil.java hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hcatalog/pig/PigHCatUtil.java new file mode 100644 index 0000000..c586cf6 --- /dev/null +++ hcatalog/hcatalog-pig-adapter/src/main/java/org/apache/hcatalog/pig/PigHCatUtil.java @@ -0,0 +1,491 @@ +/** + * 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.hcatalog.pig; + + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Properties; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.MetaStoreUtils; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hadoop.hive.ql.metadata.Table; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hcatalog.common.HCatConstants; +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.common.HCatUtil; +import org.apache.hcatalog.data.HCatRecord; +import org.apache.hcatalog.data.Pair; +import org.apache.hcatalog.data.schema.HCatFieldSchema; +import org.apache.hcatalog.data.schema.HCatFieldSchema.Type; +import org.apache.hcatalog.data.schema.HCatSchema; +import org.apache.pig.LoadPushDown.RequiredField; +import org.apache.pig.PigException; +import org.apache.pig.ResourceSchema; +import org.apache.pig.ResourceSchema.ResourceFieldSchema; +import org.apache.pig.data.DataBag; +import org.apache.pig.data.DataByteArray; +import org.apache.pig.data.DataType; +import org.apache.pig.data.DefaultDataBag; +import org.apache.pig.data.Tuple; +import org.apache.pig.data.TupleFactory; +import org.apache.pig.impl.logicalLayer.schema.Schema; +import org.apache.pig.impl.util.UDFContext; +import org.apache.pig.impl.util.Utils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.pig.PigHCatUtil} instead + */ +class PigHCatUtil { + + private static final Logger LOG = LoggerFactory.getLogger(PigHCatUtil.class); + + static final int PIG_EXCEPTION_CODE = 1115; // http://wiki.apache.org/pig/PigErrorHandlingFunctionalSpecification#Error_codes + private static final String DEFAULT_DB = MetaStoreUtils.DEFAULT_DATABASE_NAME; + + private final Map, Table> hcatTableCache = + new HashMap, Table>(); + + private static final TupleFactory tupFac = TupleFactory.getInstance(); + + private static boolean pigHasBooleanSupport = false; + + /** + * Determine if the current Pig version supports boolean columns. This works around a + * dependency conflict preventing HCatalog from requiring a version of Pig with boolean + * field support and should be removed once HCATALOG-466 has been resolved. + */ + static { + // DETAILS: + // + // PIG-1429 added support for boolean fields, which shipped in 0.10.0; + // this version of Pig depends on antlr 3.4. + // + // HCatalog depends heavily on Hive, which at this time uses antlr 3.0.1. + // + // antlr 3.0.1 and 3.4 are incompatible, so Pig 0.10.0 and Hive cannot be depended on in the + // same project. Pig 0.8.0 did not use antlr for its parser and can coexist with Hive, + // so that Pig version is depended on by HCatalog at this time. + try { + Schema schema = Utils.getSchemaFromString("myBooleanField: boolean"); + pigHasBooleanSupport = (schema.getField("myBooleanField").type == DataType.BOOLEAN); + } catch (Throwable e) { + // pass + } + + if (!pigHasBooleanSupport) { + LOG.info("This version of Pig does not support boolean fields. To enable " + + "boolean-to-integer conversion, set the " + + HCatConstants.HCAT_DATA_CONVERT_BOOLEAN_TO_INTEGER + + "=true configuration parameter."); + } + } + + static public Pair getDBTableNames(String location) throws IOException { + // the location string will be of the form: + // .
- parse it and + // communicate the information to HCatInputFormat + + try { + return HCatUtil.getDbAndTableName(location); + } catch (IOException e) { + String locationErrMsg = "The input location in load statement " + + "should be of the form " + + ".
or
. Got " + location; + throw new PigException(locationErrMsg, PIG_EXCEPTION_CODE); + } + } + + static public String getHCatServerUri(Job job) { + + return job.getConfiguration().get(HiveConf.ConfVars.METASTOREURIS.varname); + } + + static public String getHCatServerPrincipal(Job job) { + + return job.getConfiguration().get(HCatConstants.HCAT_METASTORE_PRINCIPAL); + } + + private static HiveMetaStoreClient getHiveMetaClient(String serverUri, + String serverKerberosPrincipal, Class clazz) throws Exception { + HiveConf hiveConf = new HiveConf(clazz); + + if (serverUri != null) { + hiveConf.set("hive.metastore.local", "false"); + hiveConf.setVar(HiveConf.ConfVars.METASTOREURIS, serverUri.trim()); + } + + if (serverKerberosPrincipal != null) { + hiveConf.setBoolVar(HiveConf.ConfVars.METASTORE_USE_THRIFT_SASL, true); + hiveConf.setVar(HiveConf.ConfVars.METASTORE_KERBEROS_PRINCIPAL, serverKerberosPrincipal); + } + + try { + return HCatUtil.getHiveClient(hiveConf); + } catch (Exception e) { + throw new Exception("Could not instantiate a HiveMetaStoreClient connecting to server uri:[" + serverUri + "]", e); + } + } + + + HCatSchema getHCatSchema(List fields, String signature, Class classForUDFCLookup) throws IOException { + if (fields == null) { + return null; + } + + Properties props = UDFContext.getUDFContext().getUDFProperties( + classForUDFCLookup, new String[]{signature}); + HCatSchema hcatTableSchema = (HCatSchema) props.get(HCatConstants.HCAT_TABLE_SCHEMA); + + ArrayList fcols = new ArrayList(); + for (RequiredField rf : fields) { + fcols.add(hcatTableSchema.getFields().get(rf.getIndex())); + } + return new HCatSchema(fcols); + } + + public Table getTable(String location, String hcatServerUri, String hcatServerPrincipal) throws IOException { + Pair loc_server = new Pair(location, hcatServerUri); + Table hcatTable = hcatTableCache.get(loc_server); + if (hcatTable != null) { + return hcatTable; + } + + Pair dbTablePair = PigHCatUtil.getDBTableNames(location); + String dbName = dbTablePair.first; + String tableName = dbTablePair.second; + Table table = null; + HiveMetaStoreClient client = null; + try { + client = getHiveMetaClient(hcatServerUri, hcatServerPrincipal, PigHCatUtil.class); + table = HCatUtil.getTable(client, dbName, tableName); + } catch (NoSuchObjectException nsoe) { + throw new PigException("Table not found : " + nsoe.getMessage(), PIG_EXCEPTION_CODE); // prettier error messages to frontend + } catch (Exception e) { + throw new IOException(e); + } finally { + HCatUtil.closeHiveClientQuietly(client); + } + hcatTableCache.put(loc_server, table); + return table; + } + + public static ResourceSchema getResourceSchema(HCatSchema hcatSchema) throws IOException { + + List rfSchemaList = new ArrayList(); + for (HCatFieldSchema hfs : hcatSchema.getFields()) { + ResourceFieldSchema rfSchema; + rfSchema = getResourceSchemaFromFieldSchema(hfs); + rfSchemaList.add(rfSchema); + } + ResourceSchema rSchema = new ResourceSchema(); + rSchema.setFields(rfSchemaList.toArray(new ResourceFieldSchema[0])); + return rSchema; + + } + + private static ResourceFieldSchema getResourceSchemaFromFieldSchema(HCatFieldSchema hfs) + throws IOException { + ResourceFieldSchema rfSchema; + // if we are dealing with a bag or tuple column - need to worry about subschema + if (hfs.getType() == Type.STRUCT) { + rfSchema = new ResourceFieldSchema() + .setName(hfs.getName()) + .setDescription(hfs.getComment()) + .setType(getPigType(hfs)) + .setSchema(getTupleSubSchema(hfs)); + } else if (hfs.getType() == Type.ARRAY) { + rfSchema = new ResourceFieldSchema() + .setName(hfs.getName()) + .setDescription(hfs.getComment()) + .setType(getPigType(hfs)) + .setSchema(getBagSubSchema(hfs)); + } else { + rfSchema = new ResourceFieldSchema() + .setName(hfs.getName()) + .setDescription(hfs.getComment()) + .setType(getPigType(hfs)) + .setSchema(null); // no munging inner-schemas + } + return rfSchema; + } + + protected static ResourceSchema getBagSubSchema(HCatFieldSchema hfs) throws IOException { + // there are two cases - array and array> + // in either case the element type of the array is represented in a + // tuple field schema in the bag's field schema - the second case (struct) + // more naturally translates to the tuple - in the first case (array) + // we simulate the tuple by putting the single field in a tuple + + Properties props = UDFContext.getUDFContext().getClientSystemProps(); + String innerTupleName = HCatConstants.HCAT_PIG_INNER_TUPLE_NAME_DEFAULT; + if (props != null && props.containsKey(HCatConstants.HCAT_PIG_INNER_TUPLE_NAME)) { + innerTupleName = props.getProperty(HCatConstants.HCAT_PIG_INNER_TUPLE_NAME) + .replaceAll("FIELDNAME", hfs.getName()); + } + String innerFieldName = HCatConstants.HCAT_PIG_INNER_FIELD_NAME_DEFAULT; + if (props != null && props.containsKey(HCatConstants.HCAT_PIG_INNER_FIELD_NAME)) { + innerFieldName = props.getProperty(HCatConstants.HCAT_PIG_INNER_FIELD_NAME) + .replaceAll("FIELDNAME", hfs.getName()); + } + + ResourceFieldSchema[] bagSubFieldSchemas = new ResourceFieldSchema[1]; + bagSubFieldSchemas[0] = new ResourceFieldSchema().setName(innerTupleName) + .setDescription("The tuple in the bag") + .setType(DataType.TUPLE); + HCatFieldSchema arrayElementFieldSchema = hfs.getArrayElementSchema().get(0); + if (arrayElementFieldSchema.getType() == Type.STRUCT) { + bagSubFieldSchemas[0].setSchema(getTupleSubSchema(arrayElementFieldSchema)); + } else if (arrayElementFieldSchema.getType() == Type.ARRAY) { + ResourceSchema s = new ResourceSchema(); + List lrfs = Arrays.asList(getResourceSchemaFromFieldSchema(arrayElementFieldSchema)); + s.setFields(lrfs.toArray(new ResourceFieldSchema[0])); + bagSubFieldSchemas[0].setSchema(s); + } else { + ResourceFieldSchema[] innerTupleFieldSchemas = new ResourceFieldSchema[1]; + innerTupleFieldSchemas[0] = new ResourceFieldSchema().setName(innerFieldName) + .setDescription("The inner field in the tuple in the bag") + .setType(getPigType(arrayElementFieldSchema)) + .setSchema(null); // the element type is not a tuple - so no subschema + bagSubFieldSchemas[0].setSchema(new ResourceSchema().setFields(innerTupleFieldSchemas)); + } + ResourceSchema s = new ResourceSchema().setFields(bagSubFieldSchemas); + return s; + + } + + private static ResourceSchema getTupleSubSchema(HCatFieldSchema hfs) throws IOException { + // for each struct subfield, create equivalent ResourceFieldSchema + ResourceSchema s = new ResourceSchema(); + List lrfs = new ArrayList(); + for (HCatFieldSchema subField : hfs.getStructSubSchema().getFields()) { + lrfs.add(getResourceSchemaFromFieldSchema(subField)); + } + s.setFields(lrfs.toArray(new ResourceFieldSchema[0])); + return s; + } + + /** + * @param hfs the field schema of the column + * @return corresponding pig type + * @throws IOException + */ + static public byte getPigType(HCatFieldSchema hfs) throws IOException { + return getPigType(hfs.getType()); + } + + static public byte getPigType(Type type) throws IOException { + if (type == Type.STRING) { + return DataType.CHARARRAY; + } + + if ((type == Type.INT) || (type == Type.SMALLINT) || (type == Type.TINYINT)) { + return DataType.INTEGER; + } + + if (type == Type.ARRAY) { + return DataType.BAG; + } + + if (type == Type.STRUCT) { + return DataType.TUPLE; + } + + if (type == Type.MAP) { + return DataType.MAP; + } + + if (type == Type.BIGINT) { + return DataType.LONG; + } + + if (type == Type.FLOAT) { + return DataType.FLOAT; + } + + if (type == Type.DOUBLE) { + return DataType.DOUBLE; + } + + if (type == Type.BINARY) { + return DataType.BYTEARRAY; + } + + if (type == Type.BOOLEAN && pigHasBooleanSupport) { + return DataType.BOOLEAN; + } + + throw new PigException("HCatalog column type '" + type.toString() + + "' is not supported in Pig as a column type", PIG_EXCEPTION_CODE); + } + + public static Tuple transformToTuple(HCatRecord hr, HCatSchema hs) throws Exception { + if (hr == null) { + return null; + } + return transformToTuple(hr.getAll(), hs); + } + + @SuppressWarnings("unchecked") + public static Object extractPigObject(Object o, HCatFieldSchema hfs) throws Exception { + Object result; + Type itemType = hfs.getType(); + switch (itemType) { + case BINARY: + result = (o == null) ? null : new DataByteArray((byte[]) o); + break; + case STRUCT: + result = transformToTuple((List) o, hfs); + break; + case ARRAY: + result = transformToBag((List) o, hfs); + break; + case MAP: + result = transformToPigMap((Map) o, hfs); + break; + default: + result = o; + break; + } + return result; + } + + private static Tuple transformToTuple(List objList, HCatFieldSchema hfs) throws Exception { + try { + return transformToTuple(objList, hfs.getStructSubSchema()); + } catch (Exception e) { + if (hfs.getType() != Type.STRUCT) { + throw new Exception("Expected Struct type, got " + hfs.getType(), e); + } else { + throw e; + } + } + } + + private static Tuple transformToTuple(List objList, HCatSchema hs) throws Exception { + if (objList == null) { + return null; + } + Tuple t = tupFac.newTuple(objList.size()); + List subFields = hs.getFields(); + for (int i = 0; i < subFields.size(); i++) { + t.set(i, extractPigObject(objList.get(i), subFields.get(i))); + } + return t; + } + + private static Map transformToPigMap(Map map, HCatFieldSchema hfs) throws Exception { + if (map == null) { + return null; + } + + Map result = new HashMap(); + for (Entry entry : map.entrySet()) { + // since map key for Pig has to be Strings + result.put(entry.getKey().toString(), extractPigObject(entry.getValue(), hfs.getMapValueSchema().get(0))); + } + return result; + } + + @SuppressWarnings("unchecked") + private static DataBag transformToBag(List list, HCatFieldSchema hfs) throws Exception { + if (list == null) { + return null; + } + + HCatFieldSchema elementSubFieldSchema = hfs.getArrayElementSchema().getFields().get(0); + DataBag db = new DefaultDataBag(); + for (Object o : list) { + Tuple tuple; + if (elementSubFieldSchema.getType() == Type.STRUCT) { + tuple = transformToTuple((List) o, elementSubFieldSchema); + } else { + // bags always contain tuples + tuple = tupFac.newTuple(extractPigObject(o, elementSubFieldSchema)); + } + db.add(tuple); + } + return db; + } + + + private static void validateHCatSchemaFollowsPigRules(HCatSchema tblSchema) throws PigException { + for (HCatFieldSchema hcatField : tblSchema.getFields()) { + validateHcatFieldFollowsPigRules(hcatField); + } + } + + private static void validateHcatFieldFollowsPigRules(HCatFieldSchema hcatField) throws PigException { + try { + Type hType = hcatField.getType(); + switch (hType) { + case BOOLEAN: + if (!pigHasBooleanSupport) { + throw new PigException("Incompatible type found in HCat table schema: " + + hcatField, PigHCatUtil.PIG_EXCEPTION_CODE); + } + break; + case ARRAY: + validateHCatSchemaFollowsPigRules(hcatField.getArrayElementSchema()); + break; + case STRUCT: + validateHCatSchemaFollowsPigRules(hcatField.getStructSubSchema()); + break; + case MAP: + // key is only string + if (hcatField.getMapKeyType() != Type.STRING) { + LOG.info("Converting non-String key of map " + hcatField.getName() + " from " + + hcatField.getMapKeyType() + " to String."); + } + validateHCatSchemaFollowsPigRules(hcatField.getMapValueSchema()); + break; + } + } catch (HCatException e) { + throw new PigException("Incompatible type found in hcat table schema: " + hcatField, PigHCatUtil.PIG_EXCEPTION_CODE, e); + } + } + + + public static void validateHCatTableSchemaFollowsPigRules(HCatSchema hcatTableSchema) throws IOException { + validateHCatSchemaFollowsPigRules(hcatTableSchema); + } + + public static void getConfigFromUDFProperties(Properties p, Configuration config, String propName) { + if (p.getProperty(propName) != null) { + config.set(propName, p.getProperty(propName)); + } + } + + public static void saveConfigIntoUDFProperties(Properties p, Configuration config, String propName) { + if (config.get(propName) != null) { + p.setProperty(propName, config.get(propName)); + } + } + +} diff --git hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/HCatStorerWrapper.java hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/HCatStorerWrapper.java new file mode 100644 index 0000000..1bd9bf8 --- /dev/null +++ hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/HCatStorerWrapper.java @@ -0,0 +1,73 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hcatalog.pig; + +import java.io.IOException; +import java.util.Properties; + +import org.apache.hadoop.mapreduce.Job; +import org.apache.hcatalog.common.HCatConstants; +import org.apache.pig.impl.util.UDFContext; + +/** + * This class is used to test the HCAT_PIG_STORER_EXTERNAL_LOCATION property used in HCatStorer. + * When this property is set, HCatStorer writes the output to the location it specifies. Since + * the property can only be set in the UDFContext, we need this simpler wrapper to do three things: + *
    + *
  1. save the external dir specified in the Pig script
  2. + *
  3. set the same UDFContext signature as HCatStorer
  4. + *
  5. before {@link HCatStorer#setStoreLocation(String, Job)}, set the external dir in the UDFContext.
  6. + *
+ * @deprecated Use/modify {@link org.apache.hive.hcatalog.pig.HCatStorerWrapper} instead + */ +public class HCatStorerWrapper extends HCatStorer { + + private String sign; + private String externalDir; + + public HCatStorerWrapper(String partSpecs, String schema, String externalDir) throws Exception { + super(partSpecs, schema); + this.externalDir = externalDir; + } + + public HCatStorerWrapper(String partSpecs, String externalDir) throws Exception { + super(partSpecs); + this.externalDir = externalDir; + } + + public HCatStorerWrapper(String externalDir) throws Exception{ + super(); + this.externalDir = externalDir; + } + + @Override + public void setStoreLocation(String location, Job job) throws IOException { + Properties udfProps = UDFContext.getUDFContext().getUDFProperties( + this.getClass(), new String[] { sign }); + udfProps.setProperty(HCatConstants.HCAT_PIG_STORER_EXTERNAL_LOCATION, externalDir); + super.setStoreLocation(location, job); + } + + @Override + public void setStoreFuncUDFContextSignature(String signature) { + sign = signature; + super.setStoreFuncUDFContextSignature(signature); + } +} diff --git hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/MockLoader.java hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/MockLoader.java new file mode 100644 index 0000000..4e97b4b --- /dev/null +++ hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/MockLoader.java @@ -0,0 +1,183 @@ +/** + * 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.hcatalog.pig; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.mapreduce.InputFormat; +import org.apache.hadoop.mapreduce.InputSplit; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.RecordReader; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.pig.LoadFunc; +import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigSplit; +import org.apache.pig.data.Tuple; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.pig.MockLoader} instead + */ +public class MockLoader extends LoadFunc { + private static final class MockRecordReader extends RecordReader { + @Override + public void close() throws IOException { + } + + @Override + public Object getCurrentKey() throws IOException, InterruptedException { + return "mockKey"; + } + + @Override + public Object getCurrentValue() throws IOException, InterruptedException { + return "mockValue"; + } + + @Override + public float getProgress() throws IOException, InterruptedException { + return 0.5f; + } + + @Override + public void initialize(InputSplit split, TaskAttemptContext arg1) throws IOException, + InterruptedException { + } + + @Override + public boolean nextKeyValue() throws IOException, InterruptedException { + return true; + } + } + + private static final class MockInputSplit extends InputSplit implements Writable { + private String location; + + public MockInputSplit() { + } + + public MockInputSplit(String location) { + this.location = location; + } + + @Override + public String[] getLocations() throws IOException, InterruptedException { + return new String[]{location}; + } + + @Override + public long getLength() throws IOException, InterruptedException { + return 10000000; + } + + @Override + public boolean equals(Object arg0) { + return arg0 == this; + } + + @Override + public int hashCode() { + return location.hashCode(); + } + + @Override + public void readFields(DataInput arg0) throws IOException { + location = arg0.readUTF(); + } + + @Override + public void write(DataOutput arg0) throws IOException { + arg0.writeUTF(location); + } + } + + private static final class MockInputFormat extends InputFormat { + + private final String location; + + public MockInputFormat(String location) { + this.location = location; + } + + @Override + public RecordReader createRecordReader(InputSplit arg0, TaskAttemptContext arg1) + throws IOException, InterruptedException { + return new MockRecordReader(); + } + + @Override + public List getSplits(JobContext arg0) throws IOException, InterruptedException { + return Arrays.asList(new MockInputSplit(location)); + } + } + + private static final Map> locationToData = new HashMap>(); + + public static void setData(String location, Iterable data) { + locationToData.put(location, data); + } + + private String location; + + private Iterator data; + + @Override + public String relativeToAbsolutePath(String location, Path curDir) throws IOException { + return location; + } + + @Override + public void setLocation(String location, Job job) throws IOException { + this.location = location; + if (location == null) { + throw new IOException("null location passed to MockLoader"); + } + this.data = locationToData.get(location).iterator(); + if (this.data == null) { + throw new IOException("No data configured for location: " + location); + } + } + + @Override + public Tuple getNext() throws IOException { + if (data == null) { + throw new IOException("data was not correctly initialized in MockLoader"); + } + return data.hasNext() ? data.next() : null; + } + + @Override + public InputFormat getInputFormat() throws IOException { + return new MockInputFormat(location); + } + + @Override + public void prepareToRead(RecordReader arg0, PigSplit arg1) throws IOException { + } + +} diff --git hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/MyPigStorage.java hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/MyPigStorage.java new file mode 100644 index 0000000..d26ef5a --- /dev/null +++ hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/MyPigStorage.java @@ -0,0 +1,43 @@ +/** + * 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.hcatalog.pig; + +import java.io.IOException; + +import org.apache.pig.builtin.PigStorage; +import org.apache.pig.data.Tuple; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.pig.MyPigStorage} instead + */ +public class MyPigStorage extends PigStorage { + + String arg2; + + public MyPigStorage(String arg1, String arg2) throws IOException { + super(arg1); + this.arg2 = arg2; + } + + @Override + public void putNext(Tuple t) throws IOException { + t.append(arg2); + super.putNext(t); + } +} diff --git hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/TestHCatEximLoader.java.broken hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/TestHCatEximLoader.java.broken new file mode 100644 index 0000000..238edb2 --- /dev/null +++ hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/TestHCatEximLoader.java.broken @@ -0,0 +1,352 @@ +/** + * 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.hcatalog.pig; + +import java.io.IOException; +import java.util.Iterator; +import java.util.Map; +import java.util.Properties; +import java.util.TreeMap; + +import junit.framework.TestCase; + +import org.apache.hcatalog.MiniCluster; +import org.apache.pig.ExecType; +import org.apache.pig.PigServer; +import org.apache.pig.backend.executionengine.ExecException; +import org.apache.pig.data.Tuple; +import org.apache.pig.impl.util.UDFContext; + +/** + * + * TestHCatEximLoader. Assumes Exim storer is working well + * + */ +public class TestHCatEximLoader extends TestCase { + + private static final String NONPART_TABLE = "junit_unparted"; + private static final String PARTITIONED_TABLE = "junit_parted"; + private static MiniCluster cluster = MiniCluster.buildCluster(); + + private static final String dataLocation = "/tmp/data"; + private static String fqdataLocation; + private static final String exportLocation = "/tmp/export"; + private static String fqexportLocation; + + private static Properties props; + + private void cleanup() throws IOException { + MiniCluster.deleteFile(cluster, dataLocation); + MiniCluster.deleteFile(cluster, exportLocation); + } + + @Override + protected void setUp() throws Exception { + props = new Properties(); + props.setProperty("fs.default.name", cluster.getProperties().getProperty("fs.default.name")); + System.out.println("Filesystem class : " + cluster.getFileSystem().getClass().getName() + + ", fs.default.name : " + props.getProperty("fs.default.name")); + fqdataLocation = cluster.getProperties().getProperty("fs.default.name") + dataLocation; + fqexportLocation = cluster.getProperties().getProperty("fs.default.name") + exportLocation; + System.out.println("FQ Data Location :" + fqdataLocation); + System.out.println("FQ Export Location :" + fqexportLocation); + cleanup(); + } + + @Override + protected void tearDown() throws Exception { + cleanup(); + } + + private void populateDataFile() throws IOException { + MiniCluster.deleteFile(cluster, dataLocation); + String[] input = new String[] { + "237,Krishna,01/01/1990,M,IN,TN", + "238,Kalpana,01/01/2000,F,IN,KA", + "239,Satya,01/01/2001,M,US,TN", + "240,Kavya,01/01/2002,F,US,KA" + }; + MiniCluster.createInputFile(cluster, dataLocation, input); + } + + private static class EmpDetail { + String name; + String dob; + String mf; + String country; + String state; + } + + private void assertEmpDetail(Tuple t, Map eds) throws ExecException { + assertNotNull(t); + assertEquals(6, t.size()); + + assertTrue(t.get(0).getClass() == Integer.class); + assertTrue(t.get(1).getClass() == String.class); + assertTrue(t.get(2).getClass() == String.class); + assertTrue(t.get(3).getClass() == String.class); + assertTrue(t.get(4).getClass() == String.class); + assertTrue(t.get(5).getClass() == String.class); + + EmpDetail ed = eds.remove(t.get(0)); + assertNotNull(ed); + + assertEquals(ed.name, t.get(1)); + assertEquals(ed.dob, t.get(2)); + assertEquals(ed.mf, t.get(3)); + assertEquals(ed.country, t.get(4)); + assertEquals(ed.state, t.get(5)); + } + + private void addEmpDetail(Map empDetails, int id, String name, + String dob, String mf, String country, String state) { + EmpDetail ed = new EmpDetail(); + ed.name = name; + ed.dob = dob; + ed.mf = mf; + ed.country = country; + ed.state = state; + empDetails.put(id, ed); + } + + + + private void assertEmpDetail(Tuple t, Integer id, String name, String dob, String mf) + throws ExecException { + assertNotNull(t); + assertEquals(4, t.size()); + assertTrue(t.get(0).getClass() == Integer.class); + assertTrue(t.get(1).getClass() == String.class); + assertTrue(t.get(2).getClass() == String.class); + assertTrue(t.get(3).getClass() == String.class); + + assertEquals(id, t.get(0)); + assertEquals(name, t.get(1)); + assertEquals(dob, t.get(2)); + assertEquals(mf, t.get(3)); + } + + private void assertEmpDetail(Tuple t, String mf, String name) + throws ExecException { + assertNotNull(t); + assertEquals(2, t.size()); + assertTrue(t.get(0).getClass() == String.class); + assertTrue(t.get(1).getClass() == String.class); + + assertEquals(mf, t.get(0)); + assertEquals(name, t.get(1)); + } + + + + public void testLoadNonPartTable() throws Exception { + populateDataFile(); + { + PigServer server = new PigServer(ExecType.LOCAL, props); + UDFContext.getUDFContext().setClientSystemProps(); + server.setBatchOn(); + server + .registerQuery("A = load '" + + fqdataLocation + + "' using PigStorage(',') as (emp_id:int, emp_name:chararray, emp_dob:chararray, emp_sex:chararray);"); + server.registerQuery("store A into '" + NONPART_TABLE + + "' using org.apache.hcatalog.pig.HCatEximStorer('" + fqexportLocation + "');"); + server.executeBatch(); + } + { + PigServer server = new PigServer(ExecType.LOCAL, props); + UDFContext.getUDFContext().setClientSystemProps(); + + server + .registerQuery("A = load '" + + fqexportLocation + + "' using org.apache.hcatalog.pig.HCatEximLoader();"); + Iterator XIter = server.openIterator("A"); + assertTrue(XIter.hasNext()); + Tuple t = XIter.next(); + assertEmpDetail(t, 237, "Krishna", "01/01/1990", "M"); + assertTrue(XIter.hasNext()); + t = XIter.next(); + assertEmpDetail(t, 238, "Kalpana", "01/01/2000", "F"); + assertTrue(XIter.hasNext()); + t = XIter.next(); + assertEmpDetail(t, 239, "Satya", "01/01/2001", "M"); + assertTrue(XIter.hasNext()); + t = XIter.next(); + assertEmpDetail(t, 240, "Kavya", "01/01/2002", "F"); + assertFalse(XIter.hasNext()); + } + } + + public void testLoadNonPartProjection() throws Exception { + populateDataFile(); + { + PigServer server = new PigServer(ExecType.LOCAL, props); + UDFContext.getUDFContext().setClientSystemProps(); + server.setBatchOn(); + server + .registerQuery("A = load '" + + fqdataLocation + + "' using PigStorage(',') as (emp_id:int, emp_name:chararray, emp_dob:chararray, emp_sex:chararray);"); + server.registerQuery("store A into '" + NONPART_TABLE + + "' using org.apache.hcatalog.pig.HCatEximStorer('" + fqexportLocation + "');"); + server.executeBatch(); + } + { + PigServer server = new PigServer(ExecType.LOCAL, props); + UDFContext.getUDFContext().setClientSystemProps(); + + server + .registerQuery("A = load '" + + fqexportLocation + + "' using org.apache.hcatalog.pig.HCatEximLoader();"); + server.registerQuery("B = foreach A generate emp_sex, emp_name;"); + + Iterator XIter = server.openIterator("B"); + assertTrue(XIter.hasNext()); + Tuple t = XIter.next(); + assertEmpDetail(t, "M", "Krishna"); + assertTrue(XIter.hasNext()); + t = XIter.next(); + assertEmpDetail(t, "F", "Kalpana"); + assertTrue(XIter.hasNext()); + t = XIter.next(); + assertEmpDetail(t, "M", "Satya"); + assertTrue(XIter.hasNext()); + t = XIter.next(); + assertEmpDetail(t, "F", "Kavya"); + assertFalse(XIter.hasNext()); + } + } + + + public void testLoadMultiPartTable() throws Exception { + { + populateDataFile(); + PigServer server = new PigServer(ExecType.LOCAL, props); + UDFContext.getUDFContext().setClientSystemProps(); + server.setBatchOn(); + server + .registerQuery("A = load '" + + fqdataLocation + + "' using PigStorage(',') as (emp_id:int, emp_name:chararray, emp_dob:chararray, emp_sex:chararray, emp_country:chararray, emp_state:chararray);" + ); + server.registerQuery("INTN = FILTER A BY emp_country == 'IN' AND emp_state == 'TN';"); + server.registerQuery("INKA = FILTER A BY emp_country == 'IN' AND emp_state == 'KA';"); + server.registerQuery("USTN = FILTER A BY emp_country == 'US' AND emp_state == 'TN';"); + server.registerQuery("USKA = FILTER A BY emp_country == 'US' AND emp_state == 'KA';"); + server.registerQuery("store INTN into '" + PARTITIONED_TABLE + + "' using org.apache.hcatalog.pig.HCatEximStorer('" + fqexportLocation + + "', 'emp_country=in,emp_state=tn');"); + server.registerQuery("store INKA into '" + PARTITIONED_TABLE + + "' using org.apache.hcatalog.pig.HCatEximStorer('" + fqexportLocation + + "', 'emp_country=in,emp_state=ka');"); + server.registerQuery("store USTN into '" + PARTITIONED_TABLE + + "' using org.apache.hcatalog.pig.HCatEximStorer('" + fqexportLocation + + "', 'emp_country=us,emp_state=tn');"); + server.registerQuery("store USKA into '" + PARTITIONED_TABLE + + "' using org.apache.hcatalog.pig.HCatEximStorer('" + fqexportLocation + + "', 'emp_country=us,emp_state=ka');"); + server.executeBatch(); + } + { + PigServer server = new PigServer(ExecType.LOCAL, props); + UDFContext.getUDFContext().setClientSystemProps(); + + server + .registerQuery("A = load '" + + fqexportLocation + + "' using org.apache.hcatalog.pig.HCatEximLoader() " + //+ "as (emp_id:int, emp_name:chararray, emp_dob:chararray, emp_sex:chararray, emp_country:chararray, emp_state:chararray);"); + + ";"); + + Iterator XIter = server.openIterator("A"); + + Map empDetails = new TreeMap(); + addEmpDetail(empDetails, 237, "Krishna", "01/01/1990", "M", "in", "tn"); + addEmpDetail(empDetails, 238, "Kalpana", "01/01/2000", "F", "in", "ka"); + addEmpDetail(empDetails, 239, "Satya", "01/01/2001", "M", "us", "tn"); + addEmpDetail(empDetails, 240, "Kavya", "01/01/2002", "F", "us", "ka"); + + while(XIter.hasNext()) { + Tuple t = XIter.next(); + assertNotSame(0, empDetails.size()); + assertEmpDetail(t, empDetails); + } + assertEquals(0, empDetails.size()); + } + } + + public void testLoadMultiPartFilter() throws Exception { + { + populateDataFile(); + PigServer server = new PigServer(ExecType.LOCAL, props); + UDFContext.getUDFContext().setClientSystemProps(); + server.setBatchOn(); + server + .registerQuery("A = load '" + + fqdataLocation + + "' using PigStorage(',') as (emp_id:int, emp_name:chararray, emp_dob:chararray, emp_sex:chararray, emp_country:chararray, emp_state:chararray);" + ); + server.registerQuery("INTN = FILTER A BY emp_country == 'IN' AND emp_state == 'TN';"); + server.registerQuery("INKA = FILTER A BY emp_country == 'IN' AND emp_state == 'KA';"); + server.registerQuery("USTN = FILTER A BY emp_country == 'US' AND emp_state == 'TN';"); + server.registerQuery("USKA = FILTER A BY emp_country == 'US' AND emp_state == 'KA';"); + server.registerQuery("store INTN into '" + PARTITIONED_TABLE + + "' using org.apache.hcatalog.pig.HCatEximStorer('" + fqexportLocation + + "', 'emp_country=in,emp_state=tn');"); + server.registerQuery("store INKA into '" + PARTITIONED_TABLE + + "' using org.apache.hcatalog.pig.HCatEximStorer('" + fqexportLocation + + "', 'emp_country=in,emp_state=ka');"); + server.registerQuery("store USTN into '" + PARTITIONED_TABLE + + "' using org.apache.hcatalog.pig.HCatEximStorer('" + fqexportLocation + + "', 'emp_country=us,emp_state=tn');"); + server.registerQuery("store USKA into '" + PARTITIONED_TABLE + + "' using org.apache.hcatalog.pig.HCatEximStorer('" + fqexportLocation + + "', 'emp_country=us,emp_state=ka');"); + server.executeBatch(); + } + { + PigServer server = new PigServer(ExecType.LOCAL, props); + UDFContext.getUDFContext().setClientSystemProps(); + + server + .registerQuery("A = load '" + + fqexportLocation + + "' using org.apache.hcatalog.pig.HCatEximLoader() " + + ";"); + server.registerQuery("B = filter A by emp_state == 'ka';"); + + Iterator XIter = server.openIterator("B"); + + Map empDetails = new TreeMap(); + addEmpDetail(empDetails, 238, "Kalpana", "01/01/2000", "F", "in", "ka"); + addEmpDetail(empDetails, 240, "Kavya", "01/01/2002", "F", "us", "ka"); + + while(XIter.hasNext()) { + Tuple t = XIter.next(); + assertNotSame(0, empDetails.size()); + assertEmpDetail(t, empDetails); + } + assertEquals(0, empDetails.size()); + } + } + + +} diff --git hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/TestHCatLoader.java hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/TestHCatLoader.java new file mode 100644 index 0000000..6c933f9 --- /dev/null +++ hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/TestHCatLoader.java @@ -0,0 +1,452 @@ +/** + * 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.hcatalog.pig; + +import java.io.File; +import java.io.IOException; +import java.io.RandomAccessFile; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import junit.framework.TestCase; + +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.hive.cli.CliSessionState; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.CommandNeedRetryException; +import org.apache.hadoop.hive.ql.Driver; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hcatalog.HcatTestUtils; +import org.apache.hcatalog.common.HCatConstants; +import org.apache.hcatalog.data.Pair; +import org.apache.pig.ExecType; +import org.apache.pig.PigServer; +import org.apache.pig.ResourceStatistics; +import org.apache.pig.data.DataType; +import org.apache.pig.data.Tuple; +import org.apache.pig.impl.logicalLayer.schema.Schema; +import org.apache.pig.impl.logicalLayer.schema.Schema.FieldSchema; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.pig.TestHCatLoader} instead + */ +public class TestHCatLoader extends TestCase { + private static final String TEST_DATA_DIR = System.getProperty("user.dir") + + "/build/test/data/" + TestHCatLoader.class.getCanonicalName(); + private static final String TEST_WAREHOUSE_DIR = TEST_DATA_DIR + "/warehouse"; + private static final String BASIC_FILE_NAME = TEST_DATA_DIR + "/basic.input.data"; + private static final String COMPLEX_FILE_NAME = TEST_DATA_DIR + "/complex.input.data"; + + private static final String BASIC_TABLE = "junit_unparted_basic"; + private static final String COMPLEX_TABLE = "junit_unparted_complex"; + private static final String PARTITIONED_TABLE = "junit_parted_basic"; + private static final String SPECIFIC_SIZE_TABLE = "junit_specific_size"; + private static Driver driver; + + private static int guardTestCount = 6; // ugh, instantiate using introspection in guardedSetupBeforeClass + private static boolean setupHasRun = false; + + + private static Map> basicInputData; + + protected String storageFormat() { + return "RCFILE tblproperties('hcat.isd'='org.apache.hcatalog.rcfile.RCFileInputDriver'," + + "'hcat.osd'='org.apache.hcatalog.rcfile.RCFileOutputDriver')"; + } + + private void dropTable(String tablename) throws IOException, CommandNeedRetryException { + driver.run("drop table " + tablename); + } + + private void createTable(String tablename, String schema, String partitionedBy) throws IOException, CommandNeedRetryException { + String createTable; + createTable = "create table " + tablename + "(" + schema + ") "; + if ((partitionedBy != null) && (!partitionedBy.trim().isEmpty())) { + createTable = createTable + "partitioned by (" + partitionedBy + ") "; + } + createTable = createTable + "stored as " +storageFormat(); + int retCode = driver.run(createTable).getResponseCode(); + if (retCode != 0) { + throw new IOException("Failed to create table. [" + createTable + "], return code from hive driver : [" + retCode + "]"); + } + } + + private void createTable(String tablename, String schema) throws IOException, CommandNeedRetryException { + createTable(tablename, schema, null); + } + + protected void guardedSetUpBeforeClass() throws Exception { + if (!setupHasRun) { + setupHasRun = true; + } else { + return; + } + + File f = new File(TEST_WAREHOUSE_DIR); + if (f.exists()) { + FileUtil.fullyDelete(f); + } + new File(TEST_WAREHOUSE_DIR).mkdirs(); + + HiveConf hiveConf = new HiveConf(this.getClass()); + hiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); + hiveConf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, TEST_WAREHOUSE_DIR); + driver = new Driver(hiveConf); + SessionState.start(new CliSessionState(hiveConf)); + + cleanup(); + + createTable(BASIC_TABLE, "a int, b string"); + createTable(COMPLEX_TABLE, + "name string, studentid int, " + + "contact struct, " + + "currently_registered_courses array, " + + "current_grades map, " + + "phnos array>"); + + createTable(PARTITIONED_TABLE, "a int, b string", "bkt string"); + createTable(SPECIFIC_SIZE_TABLE, "a int, b string"); + + int LOOP_SIZE = 3; + String[] input = new String[LOOP_SIZE * LOOP_SIZE]; + basicInputData = new HashMap>(); + int k = 0; + for (int i = 1; i <= LOOP_SIZE; i++) { + String si = i + ""; + for (int j = 1; j <= LOOP_SIZE; j++) { + String sj = "S" + j + "S"; + input[k] = si + "\t" + sj; + basicInputData.put(k, new Pair(i, sj)); + k++; + } + } + HcatTestUtils.createTestDataFile(BASIC_FILE_NAME, input); + HcatTestUtils.createTestDataFile(COMPLEX_FILE_NAME, + new String[]{ + //"Henry Jekyll\t42\t(415-253-6367,hjekyll@contemporary.edu.uk)\t{(PHARMACOLOGY),(PSYCHIATRY)},[PHARMACOLOGY#A-,PSYCHIATRY#B+],{(415-253-6367,cell),(408-253-6367,landline)}", + //"Edward Hyde\t1337\t(415-253-6367,anonymous@b44chan.org)\t{(CREATIVE_WRITING),(COPYRIGHT_LAW)},[CREATIVE_WRITING#A+,COPYRIGHT_LAW#D],{(415-253-6367,cell),(408-253-6367,landline)}", + } + ); + + PigServer server = new PigServer(ExecType.LOCAL); + server.setBatchOn(); + server.registerQuery("A = load '" + BASIC_FILE_NAME + "' as (a:int, b:chararray);"); + + server.registerQuery("store A into '" + BASIC_TABLE + "' using org.apache.hcatalog.pig.HCatStorer();"); + server.registerQuery("store A into '" + SPECIFIC_SIZE_TABLE + "' using org.apache.hcatalog.pig.HCatStorer();"); + server.registerQuery("B = foreach A generate a,b;"); + server.registerQuery("B2 = filter B by a < 2;"); + server.registerQuery("store B2 into '" + PARTITIONED_TABLE + "' using org.apache.hcatalog.pig.HCatStorer('bkt=0');"); + + server.registerQuery("C = foreach A generate a,b;"); + server.registerQuery("C2 = filter C by a >= 2;"); + server.registerQuery("store C2 into '" + PARTITIONED_TABLE + "' using org.apache.hcatalog.pig.HCatStorer('bkt=1');"); + + server.registerQuery("D = load '" + COMPLEX_FILE_NAME + "' as (name:chararray, studentid:int, contact:tuple(phno:chararray,email:chararray), currently_registered_courses:bag{innertup:tuple(course:chararray)}, current_grades:map[ ] , phnos :bag{innertup:tuple(phno:chararray,type:chararray)});"); + server.registerQuery("store D into '" + COMPLEX_TABLE + "' using org.apache.hcatalog.pig.HCatStorer();"); + server.executeBatch(); + + } + + private void cleanup() throws IOException, CommandNeedRetryException { + dropTable(BASIC_TABLE); + dropTable(COMPLEX_TABLE); + dropTable(PARTITIONED_TABLE); + dropTable(SPECIFIC_SIZE_TABLE); + } + + protected void guardedTearDownAfterClass() throws Exception { + guardTestCount--; + if (guardTestCount > 0) { + return; + } + cleanup(); + } + + @Override + protected void setUp() throws Exception { + guardedSetUpBeforeClass(); + } + + @Override + protected void tearDown() throws Exception { + guardedTearDownAfterClass(); + } + + public void testSchemaLoadBasic() throws IOException { + + PigServer server = new PigServer(ExecType.LOCAL); + + // test that schema was loaded correctly + server.registerQuery("X = load '" + BASIC_TABLE + "' using org.apache.hcatalog.pig.HCatLoader();"); + Schema dumpedXSchema = server.dumpSchema("X"); + List Xfields = dumpedXSchema.getFields(); + assertEquals(2, Xfields.size()); + assertTrue(Xfields.get(0).alias.equalsIgnoreCase("a")); + assertTrue(Xfields.get(0).type == DataType.INTEGER); + assertTrue(Xfields.get(1).alias.equalsIgnoreCase("b")); + assertTrue(Xfields.get(1).type == DataType.CHARARRAY); + + } + + public void testReadDataBasic() throws IOException { + PigServer server = new PigServer(ExecType.LOCAL); + + server.registerQuery("X = load '" + BASIC_TABLE + "' using org.apache.hcatalog.pig.HCatLoader();"); + Iterator XIter = server.openIterator("X"); + int numTuplesRead = 0; + while (XIter.hasNext()) { + Tuple t = XIter.next(); + assertEquals(2, t.size()); + assertTrue(t.get(0).getClass() == Integer.class); + assertTrue(t.get(1).getClass() == String.class); + assertEquals(t.get(0), basicInputData.get(numTuplesRead).first); + assertEquals(t.get(1), basicInputData.get(numTuplesRead).second); + numTuplesRead++; + } + assertEquals(basicInputData.size(), numTuplesRead); + } + + public void testSchemaLoadComplex() throws IOException { + + PigServer server = new PigServer(ExecType.LOCAL); + + // test that schema was loaded correctly + server.registerQuery("K = load '" + COMPLEX_TABLE + "' using org.apache.hcatalog.pig.HCatLoader();"); + Schema dumpedKSchema = server.dumpSchema("K"); + List Kfields = dumpedKSchema.getFields(); + assertEquals(6, Kfields.size()); + + assertEquals(DataType.CHARARRAY, Kfields.get(0).type); + assertEquals("name", Kfields.get(0).alias.toLowerCase()); + + assertEquals(DataType.INTEGER, Kfields.get(1).type); + assertEquals("studentid", Kfields.get(1).alias.toLowerCase()); + + assertEquals(DataType.TUPLE, Kfields.get(2).type); + assertEquals("contact", Kfields.get(2).alias.toLowerCase()); + { + assertNotNull(Kfields.get(2).schema); + assertTrue(Kfields.get(2).schema.getFields().size() == 2); + assertTrue(Kfields.get(2).schema.getFields().get(0).type == DataType.CHARARRAY); + assertTrue(Kfields.get(2).schema.getFields().get(0).alias.equalsIgnoreCase("phno")); + assertTrue(Kfields.get(2).schema.getFields().get(1).type == DataType.CHARARRAY); + assertTrue(Kfields.get(2).schema.getFields().get(1).alias.equalsIgnoreCase("email")); + } + assertEquals(DataType.BAG, Kfields.get(3).type); + assertEquals("currently_registered_courses", Kfields.get(3).alias.toLowerCase()); + { + assertNotNull(Kfields.get(3).schema); + assertEquals(1, Kfields.get(3).schema.getFields().size()); + assertEquals(DataType.TUPLE, Kfields.get(3).schema.getFields().get(0).type); + assertNotNull(Kfields.get(3).schema.getFields().get(0).schema); + assertEquals(1, Kfields.get(3).schema.getFields().get(0).schema.getFields().size()); + assertEquals(DataType.CHARARRAY, Kfields.get(3).schema.getFields().get(0).schema.getFields().get(0).type); + // assertEquals("course",Kfields.get(3).schema.getFields().get(0).schema.getFields().get(0).alias.toLowerCase()); + // commented out, because the name becomes "innerfield" by default - we call it "course" in pig, + // but in the metadata, it'd be anonymous, so this would be autogenerated, which is fine + } + assertEquals(DataType.MAP, Kfields.get(4).type); + assertEquals("current_grades", Kfields.get(4).alias.toLowerCase()); + assertEquals(DataType.BAG, Kfields.get(5).type); + assertEquals("phnos", Kfields.get(5).alias.toLowerCase()); + { + assertNotNull(Kfields.get(5).schema); + assertEquals(1, Kfields.get(5).schema.getFields().size()); + assertEquals(DataType.TUPLE, Kfields.get(5).schema.getFields().get(0).type); + assertNotNull(Kfields.get(5).schema.getFields().get(0).schema); + assertTrue(Kfields.get(5).schema.getFields().get(0).schema.getFields().size() == 2); + assertEquals(DataType.CHARARRAY, Kfields.get(5).schema.getFields().get(0).schema.getFields().get(0).type); + assertEquals("phno", Kfields.get(5).schema.getFields().get(0).schema.getFields().get(0).alias.toLowerCase()); + assertEquals(DataType.CHARARRAY, Kfields.get(5).schema.getFields().get(0).schema.getFields().get(1).type); + assertEquals("type", Kfields.get(5).schema.getFields().get(0).schema.getFields().get(1).alias.toLowerCase()); + } + + } + + public void testReadPartitionedBasic() throws IOException, CommandNeedRetryException { + PigServer server = new PigServer(ExecType.LOCAL); + + driver.run("select * from " + PARTITIONED_TABLE); + ArrayList valuesReadFromHiveDriver = new ArrayList(); + driver.getResults(valuesReadFromHiveDriver); + assertEquals(basicInputData.size(), valuesReadFromHiveDriver.size()); + + server.registerQuery("W = load '" + PARTITIONED_TABLE + "' using org.apache.hcatalog.pig.HCatLoader();"); + Schema dumpedWSchema = server.dumpSchema("W"); + List Wfields = dumpedWSchema.getFields(); + assertEquals(3, Wfields.size()); + assertTrue(Wfields.get(0).alias.equalsIgnoreCase("a")); + assertTrue(Wfields.get(0).type == DataType.INTEGER); + assertTrue(Wfields.get(1).alias.equalsIgnoreCase("b")); + assertTrue(Wfields.get(1).type == DataType.CHARARRAY); + assertTrue(Wfields.get(2).alias.equalsIgnoreCase("bkt")); + assertTrue(Wfields.get(2).type == DataType.CHARARRAY); + + Iterator WIter = server.openIterator("W"); + Collection> valuesRead = new ArrayList>(); + while (WIter.hasNext()) { + Tuple t = WIter.next(); + assertTrue(t.size() == 3); + assertTrue(t.get(0).getClass() == Integer.class); + assertTrue(t.get(1).getClass() == String.class); + assertTrue(t.get(2).getClass() == String.class); + valuesRead.add(new Pair((Integer) t.get(0), (String) t.get(1))); + if ((Integer) t.get(0) < 2) { + assertEquals("0", t.get(2)); + } else { + assertEquals("1", t.get(2)); + } + } + assertEquals(valuesReadFromHiveDriver.size(), valuesRead.size()); + + server.registerQuery("P1 = load '" + PARTITIONED_TABLE + "' using org.apache.hcatalog.pig.HCatLoader();"); + server.registerQuery("P1filter = filter P1 by bkt == '0';"); + Iterator P1Iter = server.openIterator("P1filter"); + int count1 = 0; + while (P1Iter.hasNext()) { + Tuple t = P1Iter.next(); + + assertEquals("0", t.get(2)); + assertEquals(1, t.get(0)); + count1++; + } + assertEquals(3, count1); + + server.registerQuery("P2 = load '" + PARTITIONED_TABLE + "' using org.apache.hcatalog.pig.HCatLoader();"); + server.registerQuery("P2filter = filter P2 by bkt == '1';"); + Iterator P2Iter = server.openIterator("P2filter"); + int count2 = 0; + while (P2Iter.hasNext()) { + Tuple t = P2Iter.next(); + + assertEquals("1", t.get(2)); + assertTrue(((Integer) t.get(0)) > 1); + count2++; + } + assertEquals(6, count2); + } + + public void testProjectionsBasic() throws IOException { + + PigServer server = new PigServer(ExecType.LOCAL); + + // projections are handled by using generate, not "as" on the Load + + server.registerQuery("Y1 = load '" + BASIC_TABLE + "' using org.apache.hcatalog.pig.HCatLoader();"); + server.registerQuery("Y2 = foreach Y1 generate a;"); + server.registerQuery("Y3 = foreach Y1 generate b,a;"); + Schema dumpedY2Schema = server.dumpSchema("Y2"); + Schema dumpedY3Schema = server.dumpSchema("Y3"); + List Y2fields = dumpedY2Schema.getFields(); + List Y3fields = dumpedY3Schema.getFields(); + assertEquals(1, Y2fields.size()); + assertEquals("a", Y2fields.get(0).alias.toLowerCase()); + assertEquals(DataType.INTEGER, Y2fields.get(0).type); + assertEquals(2, Y3fields.size()); + assertEquals("b", Y3fields.get(0).alias.toLowerCase()); + assertEquals(DataType.CHARARRAY, Y3fields.get(0).type); + assertEquals("a", Y3fields.get(1).alias.toLowerCase()); + assertEquals(DataType.INTEGER, Y3fields.get(1).type); + + int numTuplesRead = 0; + Iterator Y2Iter = server.openIterator("Y2"); + while (Y2Iter.hasNext()) { + Tuple t = Y2Iter.next(); + assertEquals(t.size(), 1); + assertTrue(t.get(0).getClass() == Integer.class); + assertEquals(t.get(0), basicInputData.get(numTuplesRead).first); + numTuplesRead++; + } + numTuplesRead = 0; + Iterator Y3Iter = server.openIterator("Y3"); + while (Y3Iter.hasNext()) { + Tuple t = Y3Iter.next(); + assertEquals(t.size(), 2); + assertTrue(t.get(0).getClass() == String.class); + assertEquals(t.get(0), basicInputData.get(numTuplesRead).second); + assertTrue(t.get(1).getClass() == Integer.class); + assertEquals(t.get(1), basicInputData.get(numTuplesRead).first); + numTuplesRead++; + } + assertEquals(basicInputData.size(), numTuplesRead); + } + + public void testGetInputBytes() throws Exception { + File file = new File(TEST_WAREHOUSE_DIR + "/" + SPECIFIC_SIZE_TABLE + "/part-m-00000"); + file.deleteOnExit(); + RandomAccessFile randomAccessFile = new RandomAccessFile(file, "rw"); + randomAccessFile.setLength(2L * 1024 * 1024 * 1024); + + Job job = new Job(); + HCatLoader hCatLoader = new HCatLoader(); + hCatLoader.setUDFContextSignature(this.getName()); + hCatLoader.setLocation(SPECIFIC_SIZE_TABLE, job); + ResourceStatistics statistics = hCatLoader.getStatistics(file.getAbsolutePath(), job); + assertEquals(2048, (long) statistics.getmBytes()); + } + + public void testConvertBooleanToInt() throws Exception { + String tbl = "test_convert_boolean_to_int"; + String inputFileName = TEST_DATA_DIR + "/testConvertBooleanToInt/data.txt"; + File inputDataDir = new File(inputFileName).getParentFile(); + inputDataDir.mkdir(); + + String[] lines = new String[]{"llama\t1", "alpaca\t0"}; + HcatTestUtils.createTestDataFile(inputFileName, lines); + + assertEquals(0, driver.run("drop table if exists " + tbl).getResponseCode()); + assertEquals(0, driver.run("create external table " + tbl + + " (a string, b boolean) row format delimited fields terminated by '\t'" + + " stored as textfile location 'file://" + + inputDataDir.getAbsolutePath() + "'").getResponseCode()); + + Properties properties = new Properties(); + properties.setProperty(HCatConstants.HCAT_DATA_CONVERT_BOOLEAN_TO_INTEGER, "true"); + PigServer server = new PigServer(ExecType.LOCAL, properties); + server.registerQuery( + "data = load 'test_convert_boolean_to_int' using org.apache.hcatalog.pig.HCatLoader();"); + Schema schema = server.dumpSchema("data"); + assertEquals(2, schema.getFields().size()); + + assertEquals("a", schema.getField(0).alias); + assertEquals(DataType.CHARARRAY, schema.getField(0).type); + assertEquals("b", schema.getField(1).alias); + assertEquals(DataType.INTEGER, schema.getField(1).type); + + Iterator iterator = server.openIterator("data"); + Tuple t = iterator.next(); + assertEquals("llama", t.get(0)); + // TODO: Figure out how to load a text file into Hive with boolean columns. This next assert + // passes because data was loaded as integers, not because it was converted. + assertEquals(1, t.get(1)); + t = iterator.next(); + assertEquals("alpaca", t.get(0)); + assertEquals(0, t.get(1)); + assertFalse(iterator.hasNext()); + } +} diff --git hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/TestHCatLoaderComplexSchema.java hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/TestHCatLoaderComplexSchema.java new file mode 100644 index 0000000..64fc05c --- /dev/null +++ hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/TestHCatLoaderComplexSchema.java @@ -0,0 +1,303 @@ +/** + * 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.hcatalog.pig; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; + +import junit.framework.Assert; + +import org.apache.hadoop.hive.cli.CliSessionState; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.CommandNeedRetryException; +import org.apache.hadoop.hive.ql.Driver; +import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.pig.ExecType; +import org.apache.pig.PigServer; +import org.apache.pig.backend.executionengine.ExecException; +import org.apache.pig.backend.executionengine.ExecJob; +import org.apache.pig.data.BagFactory; +import org.apache.pig.data.DataBag; +import org.apache.pig.data.Tuple; +import org.apache.pig.data.TupleFactory; +import org.apache.pig.impl.logicalLayer.FrontendException; +import org.apache.pig.impl.logicalLayer.schema.Schema; +import org.apache.pig.impl.logicalLayer.schema.Schema.FieldSchema; +import org.junit.BeforeClass; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.pig.TestHCatLoaderComplexSchema} instead + */ +public class TestHCatLoaderComplexSchema { + + //private static MiniCluster cluster = MiniCluster.buildCluster(); + private static Driver driver; + //private static Properties props; + private static final Logger LOG = LoggerFactory.getLogger(TestHCatLoaderComplexSchema.class); + + private void dropTable(String tablename) throws IOException, CommandNeedRetryException { + driver.run("drop table " + tablename); + } + + private void createTable(String tablename, String schema, String partitionedBy) throws IOException, CommandNeedRetryException { + String createTable; + createTable = "create table " + tablename + "(" + schema + ") "; + if ((partitionedBy != null) && (!partitionedBy.trim().isEmpty())) { + createTable = createTable + "partitioned by (" + partitionedBy + ") "; + } + createTable = createTable + "stored as RCFILE tblproperties('hcat.isd'='org.apache.hcatalog.rcfile.RCFileInputDriver'," + + "'hcat.osd'='org.apache.hcatalog.rcfile.RCFileOutputDriver') "; + LOG.info("Creating table:\n {}", createTable); + CommandProcessorResponse result = driver.run(createTable); + int retCode = result.getResponseCode(); + if (retCode != 0) { + throw new IOException("Failed to create table. [" + createTable + "], return code from hive driver : [" + retCode + " " + result.getErrorMessage() + "]"); + } + } + + private void createTable(String tablename, String schema) throws IOException, CommandNeedRetryException { + createTable(tablename, schema, null); + } + + @BeforeClass + public static void setUpBeforeClass() throws Exception { + + HiveConf hiveConf = new HiveConf(TestHCatLoaderComplexSchema.class); + hiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); + driver = new Driver(hiveConf); + SessionState.start(new CliSessionState(hiveConf)); + //props = new Properties(); + //props.setProperty("fs.default.name", cluster.getProperties().getProperty("fs.default.name")); + + } + + private static final TupleFactory tf = TupleFactory.getInstance(); + private static final BagFactory bf = BagFactory.getInstance(); + + private Tuple t(Object... objects) { + return tf.newTuple(Arrays.asList(objects)); + } + + private DataBag b(Tuple... objects) { + return bf.newDefaultBag(Arrays.asList(objects)); + } + + /** + * artificially complex nested schema to test nested schema conversion + * @throws Exception + */ + @Test + public void testSyntheticComplexSchema() throws Exception { + String pigSchema = + "a: " + + "(" + + "aa: chararray, " + + "ab: long, " + + "ac: map[], " + + "ad: { t: (ada: long) }, " + + "ae: { t: (aea:long, aeb: ( aeba: chararray, aebb: long)) }," + + "af: (afa: chararray, afb: long) " + + ")," + + "b: chararray, " + + "c: long, " + + "d: { t: (da:long, db: ( dba: chararray, dbb: long), dc: { t: (dca: long) } ) } "; + + // with extra structs + String tableSchema = + "a struct<" + + "aa: string, " + + "ab: bigint, " + + "ac: map, " + + "ad: array>, " + + "ae: array>>," + + "af: struct " + + ">, " + + "b string, " + + "c bigint, " + + "d array, dc: array>>>"; + + // without extra structs + String tableSchema2 = + "a struct<" + + "aa: string, " + + "ab: bigint, " + + "ac: map, " + + "ad: array, " + + "ae: array>>," + + "af: struct " + + ">, " + + "b string, " + + "c bigint, " + + "d array, dc: array>>"; + + List data = new ArrayList(); + for (int i = 0; i < 10; i++) { + Tuple t = t( + t( + "aa test", + 2l, + new HashMap() { + { + put("ac test1", "test 1"); + put("ac test2", "test 2"); + } + }, + b(t(3l), t(4l)), + b(t(5l, t("aeba test", 6l))), + t("afa test", 7l) + ), + "b test", + (long) i, + b(t(8l, t("dba test", 9l), b(t(10l))))); + + data.add(t); + } + verifyWriteRead("testSyntheticComplexSchema", pigSchema, tableSchema, data, true); + verifyWriteRead("testSyntheticComplexSchema", pigSchema, tableSchema, data, false); + verifyWriteRead("testSyntheticComplexSchema2", pigSchema, tableSchema2, data, true); + verifyWriteRead("testSyntheticComplexSchema2", pigSchema, tableSchema2, data, false); + + } + + private void verifyWriteRead(String tablename, String pigSchema, String tableSchema, List data, boolean provideSchemaToStorer) + throws IOException, CommandNeedRetryException, ExecException, FrontendException { + MockLoader.setData(tablename + "Input", data); + try { + createTable(tablename, tableSchema); + PigServer server = new PigServer(ExecType.LOCAL); + server.setBatchOn(); + server.registerQuery("A = load '" + tablename + "Input' using org.apache.hcatalog.pig.MockLoader() AS (" + pigSchema + ");"); + Schema dumpedASchema = server.dumpSchema("A"); + server.registerQuery("STORE A into '" + tablename + "' using org.apache.hcatalog.pig.HCatStorer(" + + (provideSchemaToStorer ? "'', '" + pigSchema + "'" : "") + + ");"); + + ExecJob execJob = server.executeBatch().get(0); + if (!execJob.getStatistics().isSuccessful()) { + throw new RuntimeException("Import failed", execJob.getException()); + } + // test that schema was loaded correctly + server.registerQuery("X = load '" + tablename + "' using org.apache.hcatalog.pig.HCatLoader();"); + server.dumpSchema("X"); + Iterator it = server.openIterator("X"); + int i = 0; + while (it.hasNext()) { + Tuple input = data.get(i++); + Tuple output = it.next(); + Assert.assertEquals(input.toString(), output.toString()); + LOG.info("tuple : {} ", output); + } + Schema dumpedXSchema = server.dumpSchema("X"); + + Assert.assertEquals( + "expected " + dumpedASchema + " but was " + dumpedXSchema + " (ignoring field names)", + "", + compareIgnoreFiledNames(dumpedASchema, dumpedXSchema)); + + } finally { + dropTable(tablename); + } + } + + private String compareIgnoreFiledNames(Schema expected, Schema got) throws FrontendException { + if (expected == null || got == null) { + if (expected == got) { + return ""; + } else { + return "\nexpected " + expected + " got " + got; + } + } + if (expected.size() != got.size()) { + return "\nsize expected " + expected.size() + " (" + expected + ") got " + got.size() + " (" + got + ")"; + } + String message = ""; + for (int i = 0; i < expected.size(); i++) { + FieldSchema expectedField = expected.getField(i); + FieldSchema gotField = got.getField(i); + if (expectedField.type != gotField.type) { + message += "\ntype expected " + expectedField.type + " (" + expectedField + ") got " + gotField.type + " (" + gotField + ")"; + } else { + message += compareIgnoreFiledNames(expectedField.schema, gotField.schema); + } + } + return message; + } + + /** + * tests that unnecessary tuples are drop while converting schema + * (Pig requires Tuples in Bags) + * @throws Exception + */ + @Test + public void testTupleInBagInTupleInBag() throws Exception { + String pigSchema = "a: { b : ( c: { d: (i : long) } ) }"; + + String tableSchema = "a array< array< bigint > >"; + + List data = new ArrayList(); + data.add(t(b(t(b(t(100l), t(101l))), t(b(t(110l)))))); + data.add(t(b(t(b(t(200l))), t(b(t(210l))), t(b(t(220l)))))); + data.add(t(b(t(b(t(300l), t(301l)))))); + data.add(t(b(t(b(t(400l))), t(b(t(410l), t(411l), t(412l)))))); + + + verifyWriteRead("TupleInBagInTupleInBag1", pigSchema, tableSchema, data, true); + verifyWriteRead("TupleInBagInTupleInBag2", pigSchema, tableSchema, data, false); + + // test that we don't drop the unnecessary tuple if the table has the corresponding Struct + String tableSchema2 = "a array< struct< c: array< struct< i: bigint > > > >"; + + verifyWriteRead("TupleInBagInTupleInBag3", pigSchema, tableSchema2, data, true); + verifyWriteRead("TupleInBagInTupleInBag4", pigSchema, tableSchema2, data, false); + + } + + @Test + public void testMapWithComplexData() throws Exception { + String pigSchema = "a: long, b: map[]"; + String tableSchema = "a bigint, b map>"; + + List data = new ArrayList(); + for (int i = 0; i < 10; i++) { + Tuple t = t( + (long) i, + new HashMap() { + { + put("b test 1", t(1l, "test 1")); + put("b test 2", t(2l, "test 2")); + } + }); + + data.add(t); + } + verifyWriteRead("testMapWithComplexData", pigSchema, tableSchema, data, true); + verifyWriteRead("testMapWithComplexData2", pigSchema, tableSchema, data, false); + + } +} diff --git hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/TestHCatLoaderStorer.java hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/TestHCatLoaderStorer.java new file mode 100644 index 0000000..0cab16c --- /dev/null +++ hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/TestHCatLoaderStorer.java @@ -0,0 +1,142 @@ +/** + * 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.hcatalog.pig; + +import org.apache.hadoop.fs.FileUtil; +import org.apache.hcatalog.HcatTestUtils; +import org.apache.hcatalog.mapreduce.HCatBaseTest; +import org.apache.pig.ExecType; +import org.apache.pig.PigServer; +import org.apache.pig.backend.executionengine.ExecJob; +import org.apache.pig.data.DataType; +import org.apache.pig.data.Tuple; +import org.apache.pig.impl.logicalLayer.schema.Schema; +import org.junit.Assert; +import org.junit.Test; + +import java.io.File; +import java.util.Iterator; +import java.util.List; + +/** + * Test that require both HCatLoader and HCatStorer. For read or write only functionality, + * please consider @{link TestHCatLoader} or @{link TestHCatStorer}. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.pig.TestHCatLoaderStorer} instead + */ +public class TestHCatLoaderStorer extends HCatBaseTest { + + /** + * Ensure Pig can read/write tinyint/smallint columns. + */ + @Test + public void testSmallTinyInt() throws Exception { + + String readTblName = "test_small_tiny_int"; + File dataDir = new File(TEST_DATA_DIR + "/testSmallTinyIntData"); + File dataFile = new File(dataDir, "testSmallTinyInt.tsv"); + + String writeTblName = "test_small_tiny_int_write"; + File writeDataFile = new File(TEST_DATA_DIR, writeTblName + ".tsv"); + + FileUtil.fullyDelete(dataDir); // Might not exist + Assert.assertTrue(dataDir.mkdir()); + + HcatTestUtils.createTestDataFile(dataFile.getAbsolutePath(), new String[]{ + String.format("%d\t%d", Short.MIN_VALUE, Byte.MIN_VALUE), + String.format("%d\t%d", Short.MAX_VALUE, Byte.MAX_VALUE) + }); + + // Create a table with smallint/tinyint columns, load data, and query from Hive. + Assert.assertEquals(0, driver.run("drop table if exists " + readTblName).getResponseCode()); + Assert.assertEquals(0, driver.run("create external table " + readTblName + + " (my_small_int smallint, my_tiny_int tinyint)" + + " row format delimited fields terminated by '\t' stored as textfile").getResponseCode()); + Assert.assertEquals(0, driver.run("load data local inpath '" + + dataDir.getAbsolutePath() + "' into table " + readTblName).getResponseCode()); + + PigServer server = new PigServer(ExecType.LOCAL); + server.registerQuery( + "data = load '" + readTblName + "' using org.apache.hcatalog.pig.HCatLoader();"); + + // Ensure Pig schema is correct. + Schema schema = server.dumpSchema("data"); + Assert.assertEquals(2, schema.getFields().size()); + Assert.assertEquals("my_small_int", schema.getField(0).alias); + Assert.assertEquals(DataType.INTEGER, schema.getField(0).type); + Assert.assertEquals("my_tiny_int", schema.getField(1).alias); + Assert.assertEquals(DataType.INTEGER, schema.getField(1).type); + + // Ensure Pig can read data correctly. + Iterator it = server.openIterator("data"); + Tuple t = it.next(); + Assert.assertEquals(new Integer(Short.MIN_VALUE), t.get(0)); + Assert.assertEquals(new Integer(Byte.MIN_VALUE), t.get(1)); + t = it.next(); + Assert.assertEquals(new Integer(Short.MAX_VALUE), t.get(0)); + Assert.assertEquals(new Integer(Byte.MAX_VALUE), t.get(1)); + Assert.assertFalse(it.hasNext()); + + // Ensure Pig can write correctly to smallint/tinyint columns. This means values within the + // bounds of the column type are written, and values outside throw an exception. + Assert.assertEquals(0, driver.run("drop table if exists " + writeTblName).getResponseCode()); + Assert.assertEquals(0, driver.run("create table " + writeTblName + + " (my_small_int smallint, my_tiny_int tinyint) stored as rcfile").getResponseCode()); + + // Values within the column type bounds. + HcatTestUtils.createTestDataFile(writeDataFile.getAbsolutePath(), new String[]{ + String.format("%d\t%d", Short.MIN_VALUE, Byte.MIN_VALUE), + String.format("%d\t%d", Short.MAX_VALUE, Byte.MAX_VALUE) + }); + smallTinyIntBoundsCheckHelper(writeDataFile.getAbsolutePath(), ExecJob.JOB_STATUS.COMPLETED); + + // Values outside the column type bounds will fail at runtime. + HcatTestUtils.createTestDataFile(TEST_DATA_DIR + "/shortTooSmall.tsv", new String[]{ + String.format("%d\t%d", Short.MIN_VALUE - 1, 0)}); + smallTinyIntBoundsCheckHelper(TEST_DATA_DIR + "/shortTooSmall.tsv", ExecJob.JOB_STATUS.FAILED); + + HcatTestUtils.createTestDataFile(TEST_DATA_DIR + "/shortTooBig.tsv", new String[]{ + String.format("%d\t%d", Short.MAX_VALUE + 1, 0)}); + smallTinyIntBoundsCheckHelper(TEST_DATA_DIR + "/shortTooBig.tsv", ExecJob.JOB_STATUS.FAILED); + + HcatTestUtils.createTestDataFile(TEST_DATA_DIR + "/byteTooSmall.tsv", new String[]{ + String.format("%d\t%d", 0, Byte.MIN_VALUE - 1)}); + smallTinyIntBoundsCheckHelper(TEST_DATA_DIR + "/byteTooSmall.tsv", ExecJob.JOB_STATUS.FAILED); + + HcatTestUtils.createTestDataFile(TEST_DATA_DIR + "/byteTooBig.tsv", new String[]{ + String.format("%d\t%d", 0, Byte.MAX_VALUE + 1)}); + smallTinyIntBoundsCheckHelper(TEST_DATA_DIR + "/byteTooBig.tsv", ExecJob.JOB_STATUS.FAILED); + } + + private void smallTinyIntBoundsCheckHelper(String data, ExecJob.JOB_STATUS expectedStatus) + throws Exception { + Assert.assertEquals(0, driver.run("drop table if exists test_tbl").getResponseCode()); + Assert.assertEquals(0, driver.run("create table test_tbl" + + " (my_small_int smallint, my_tiny_int tinyint) stored as rcfile").getResponseCode()); + + PigServer server = new PigServer(ExecType.LOCAL); + server.setBatchOn(); + server.registerQuery("data = load '" + data + + "' using PigStorage('\t') as (my_small_int:int, my_tiny_int:int);"); + server.registerQuery( + "store data into 'test_tbl' using org.apache.hcatalog.pig.HCatStorer();"); + List jobs = server.executeBatch(); + Assert.assertEquals(expectedStatus, jobs.get(0).getStatus()); + } +} diff --git hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/TestHCatStorer.java hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/TestHCatStorer.java new file mode 100644 index 0000000..0bf898b --- /dev/null +++ hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/TestHCatStorer.java @@ -0,0 +1,660 @@ +/** + * 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.hcatalog.pig; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; + +import org.apache.hadoop.hive.ql.CommandNeedRetryException; +import org.apache.hcatalog.HcatTestUtils; +import org.apache.hcatalog.mapreduce.HCatBaseTest; +import org.apache.pig.EvalFunc; +import org.apache.pig.ExecType; +import org.apache.pig.PigException; +import org.apache.pig.PigServer; +import org.apache.pig.data.DataByteArray; +import org.apache.pig.data.Tuple; +import org.apache.pig.impl.logicalLayer.FrontendException; +import org.apache.pig.impl.util.LogUtils; +import org.junit.Assert; +import org.junit.Test; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.pig.TestHCatStorer} instead + */ +public class TestHCatStorer extends HCatBaseTest { + + private static final String INPUT_FILE_NAME = TEST_DATA_DIR + "/input.data"; + + @Test + public void testPartColsInData() throws IOException, CommandNeedRetryException { + + driver.run("drop table junit_unparted"); + String createTable = "create table junit_unparted(a int) partitioned by (b string) stored as RCFILE"; + int retCode = driver.run(createTable).getResponseCode(); + if (retCode != 0) { + throw new IOException("Failed to create table."); + } + int LOOP_SIZE = 11; + String[] input = new String[LOOP_SIZE]; + for (int i = 0; i < LOOP_SIZE; i++) { + input[i] = i + "\t1"; + } + HcatTestUtils.createTestDataFile(INPUT_FILE_NAME, input); + PigServer server = new PigServer(ExecType.LOCAL); + server.registerQuery("A = load '" + INPUT_FILE_NAME + "' as (a:int, b:chararray);"); + server.registerQuery("store A into 'default.junit_unparted' using " + HCatStorer.class.getName() + "('b=1');"); + server.registerQuery("B = load 'default.junit_unparted' using " + HCatLoader.class.getName() + "();"); + Iterator itr = server.openIterator("B"); + + int i = 0; + + while (itr.hasNext()) { + Tuple t = itr.next(); + Assert.assertEquals(2, t.size()); + Assert.assertEquals(t.get(0), i); + Assert.assertEquals(t.get(1), "1"); + i++; + } + + Assert.assertFalse(itr.hasNext()); + Assert.assertEquals(11, i); + } + + @Test + public void testMultiPartColsInData() throws IOException, CommandNeedRetryException { + + driver.run("drop table employee"); + String createTable = "CREATE TABLE employee (emp_id INT, emp_name STRING, emp_start_date STRING , emp_gender STRING ) " + + " PARTITIONED BY (emp_country STRING , emp_state STRING ) STORED AS RCFILE"; + + int retCode = driver.run(createTable).getResponseCode(); + if (retCode != 0) { + throw new IOException("Failed to create table."); + } + + String[] inputData = {"111237\tKrishna\t01/01/1990\tM\tIN\tTN", + "111238\tKalpana\t01/01/2000\tF\tIN\tKA", + "111239\tSatya\t01/01/2001\tM\tIN\tKL", + "111240\tKavya\t01/01/2002\tF\tIN\tAP"}; + + HcatTestUtils.createTestDataFile(INPUT_FILE_NAME, inputData); + PigServer pig = new PigServer(ExecType.LOCAL); + pig.setBatchOn(); + pig.registerQuery("A = LOAD '" + INPUT_FILE_NAME + "' USING PigStorage() AS (emp_id:int,emp_name:chararray,emp_start_date:chararray," + + "emp_gender:chararray,emp_country:chararray,emp_state:chararray);"); + pig.registerQuery("TN = FILTER A BY emp_state == 'TN';"); + pig.registerQuery("KA = FILTER A BY emp_state == 'KA';"); + pig.registerQuery("KL = FILTER A BY emp_state == 'KL';"); + pig.registerQuery("AP = FILTER A BY emp_state == 'AP';"); + pig.registerQuery("STORE TN INTO 'employee' USING " + HCatStorer.class.getName() + "('emp_country=IN,emp_state=TN');"); + pig.registerQuery("STORE KA INTO 'employee' USING " + HCatStorer.class.getName() + "('emp_country=IN,emp_state=KA');"); + pig.registerQuery("STORE KL INTO 'employee' USING " + HCatStorer.class.getName() + "('emp_country=IN,emp_state=KL');"); + pig.registerQuery("STORE AP INTO 'employee' USING " + HCatStorer.class.getName() + "('emp_country=IN,emp_state=AP');"); + pig.executeBatch(); + driver.run("select * from employee"); + ArrayList results = new ArrayList(); + driver.getResults(results); + Assert.assertEquals(4, results.size()); + Collections.sort(results); + Assert.assertEquals(inputData[0], results.get(0)); + Assert.assertEquals(inputData[1], results.get(1)); + Assert.assertEquals(inputData[2], results.get(2)); + Assert.assertEquals(inputData[3], results.get(3)); + driver.run("drop table employee"); + } + + @Test + public void testStoreInPartiitonedTbl() throws IOException, CommandNeedRetryException { + + driver.run("drop table junit_unparted"); + String createTable = "create table junit_unparted(a int) partitioned by (b string) stored as RCFILE"; + int retCode = driver.run(createTable).getResponseCode(); + if (retCode != 0) { + throw new IOException("Failed to create table."); + } + int LOOP_SIZE = 11; + String[] input = new String[LOOP_SIZE]; + for (int i = 0; i < LOOP_SIZE; i++) { + input[i] = i + ""; + } + HcatTestUtils.createTestDataFile(INPUT_FILE_NAME, input); + PigServer server = new PigServer(ExecType.LOCAL); + server.registerQuery("A = load '" + INPUT_FILE_NAME + "' as (a:int);"); + server.registerQuery("store A into 'default.junit_unparted' using " + HCatStorer.class.getName() + "('b=1');"); + server.registerQuery("B = load 'default.junit_unparted' using " + HCatLoader.class.getName() + "();"); + Iterator itr = server.openIterator("B"); + + int i = 0; + + while (itr.hasNext()) { + Tuple t = itr.next(); + Assert.assertEquals(2, t.size()); + Assert.assertEquals(t.get(0), i); + Assert.assertEquals(t.get(1), "1"); + i++; + } + + Assert.assertFalse(itr.hasNext()); + Assert.assertEquals(11, i); + } + + @Test + public void testNoAlias() throws IOException, CommandNeedRetryException { + driver.run("drop table junit_parted"); + String createTable = "create table junit_parted(a int, b string) partitioned by (ds string) stored as RCFILE"; + int retCode = driver.run(createTable).getResponseCode(); + if (retCode != 0) { + throw new IOException("Failed to create table."); + } + PigServer server = new PigServer(ExecType.LOCAL); + boolean errCaught = false; + try { + server.setBatchOn(); + server.registerQuery("A = load '" + INPUT_FILE_NAME + "' as (a:int, b:chararray);"); + server.registerQuery("B = foreach A generate a+10, b;"); + server.registerQuery("store B into 'junit_parted' using " + HCatStorer.class.getName() + "('ds=20100101');"); + server.executeBatch(); + } catch (PigException fe) { + PigException pe = LogUtils.getPigException(fe); + Assert.assertTrue(pe instanceof FrontendException); + Assert.assertEquals(PigHCatUtil.PIG_EXCEPTION_CODE, pe.getErrorCode()); + Assert.assertTrue(pe.getMessage().contains("Column name for a field is not specified. Please provide the full schema as an argument to HCatStorer.")); + errCaught = true; + } + Assert.assertTrue(errCaught); + errCaught = false; + try { + server.setBatchOn(); + server.registerQuery("A = load '" + INPUT_FILE_NAME + "' as (a:int, B:chararray);"); + server.registerQuery("B = foreach A generate a, B;"); + server.registerQuery("store B into 'junit_parted' using " + HCatStorer.class.getName() + "('ds=20100101');"); + server.executeBatch(); + } catch (PigException fe) { + PigException pe = LogUtils.getPigException(fe); + Assert.assertTrue(pe instanceof FrontendException); + Assert.assertEquals(PigHCatUtil.PIG_EXCEPTION_CODE, pe.getErrorCode()); + Assert.assertTrue(pe.getMessage().contains("Column names should all be in lowercase. Invalid name found: B")); + errCaught = true; + } + driver.run("drop table junit_parted"); + Assert.assertTrue(errCaught); + } + + @Test + public void testStoreMultiTables() throws IOException, CommandNeedRetryException { + + driver.run("drop table junit_unparted"); + String createTable = "create table junit_unparted(a int, b string) stored as RCFILE"; + int retCode = driver.run(createTable).getResponseCode(); + if (retCode != 0) { + throw new IOException("Failed to create table."); + } + driver.run("drop table junit_unparted2"); + createTable = "create table junit_unparted2(a int, b string) stored as RCFILE"; + retCode = driver.run(createTable).getResponseCode(); + if (retCode != 0) { + throw new IOException("Failed to create table."); + } + + int LOOP_SIZE = 3; + String[] input = new String[LOOP_SIZE * LOOP_SIZE]; + int k = 0; + for (int i = 1; i <= LOOP_SIZE; i++) { + String si = i + ""; + for (int j = 1; j <= LOOP_SIZE; j++) { + input[k++] = si + "\t" + j; + } + } + HcatTestUtils.createTestDataFile(INPUT_FILE_NAME, input); + PigServer server = new PigServer(ExecType.LOCAL); + server.setBatchOn(); + server.registerQuery("A = load '" + INPUT_FILE_NAME + "' as (a:int, b:chararray);"); + server.registerQuery("B = filter A by a < 2;"); + server.registerQuery("store B into 'junit_unparted' using " + HCatStorer.class.getName() + "();"); + server.registerQuery("C = filter A by a >= 2;"); + server.registerQuery("store C into 'junit_unparted2' using " + HCatStorer.class.getName() + "();"); + server.executeBatch(); + + driver.run("select * from junit_unparted"); + ArrayList res = new ArrayList(); + driver.getResults(res); + driver.run("select * from junit_unparted2"); + ArrayList res2 = new ArrayList(); + driver.getResults(res2); + + res.addAll(res2); + driver.run("drop table junit_unparted"); + driver.run("drop table junit_unparted2"); + + Iterator itr = res.iterator(); + for (int i = 0; i < LOOP_SIZE * LOOP_SIZE; i++) { + Assert.assertEquals(input[i], itr.next()); + } + + Assert.assertFalse(itr.hasNext()); + + } + + @Test + public void testStoreWithNoSchema() throws IOException, CommandNeedRetryException { + + driver.run("drop table junit_unparted"); + String createTable = "create table junit_unparted(a int, b string) stored as RCFILE"; + int retCode = driver.run(createTable).getResponseCode(); + if (retCode != 0) { + throw new IOException("Failed to create table."); + } + + int LOOP_SIZE = 3; + String[] input = new String[LOOP_SIZE * LOOP_SIZE]; + int k = 0; + for (int i = 1; i <= LOOP_SIZE; i++) { + String si = i + ""; + for (int j = 1; j <= LOOP_SIZE; j++) { + input[k++] = si + "\t" + j; + } + } + HcatTestUtils.createTestDataFile(INPUT_FILE_NAME, input); + PigServer server = new PigServer(ExecType.LOCAL); + server.setBatchOn(); + server.registerQuery("A = load '" + INPUT_FILE_NAME + "' as (a:int, b:chararray);"); + server.registerQuery("store A into 'default.junit_unparted' using " + HCatStorer.class.getName() + "('');"); + server.executeBatch(); + + driver.run("select * from junit_unparted"); + ArrayList res = new ArrayList(); + driver.getResults(res); + driver.run("drop table junit_unparted"); + Iterator itr = res.iterator(); + for (int i = 0; i < LOOP_SIZE * LOOP_SIZE; i++) { + Assert.assertEquals(input[i], itr.next()); + } + + Assert.assertFalse(itr.hasNext()); + + } + + @Test + public void testStoreWithNoCtorArgs() throws IOException, CommandNeedRetryException { + + driver.run("drop table junit_unparted"); + String createTable = "create table junit_unparted(a int, b string) stored as RCFILE"; + int retCode = driver.run(createTable).getResponseCode(); + if (retCode != 0) { + throw new IOException("Failed to create table."); + } + + int LOOP_SIZE = 3; + String[] input = new String[LOOP_SIZE * LOOP_SIZE]; + int k = 0; + for (int i = 1; i <= LOOP_SIZE; i++) { + String si = i + ""; + for (int j = 1; j <= LOOP_SIZE; j++) { + input[k++] = si + "\t" + j; + } + } + HcatTestUtils.createTestDataFile(INPUT_FILE_NAME, input); + PigServer server = new PigServer(ExecType.LOCAL); + server.setBatchOn(); + server.registerQuery("A = load '" + INPUT_FILE_NAME + "' as (a:int, b:chararray);"); + server.registerQuery("store A into 'junit_unparted' using " + HCatStorer.class.getName() + "();"); + server.executeBatch(); + + driver.run("select * from junit_unparted"); + ArrayList res = new ArrayList(); + driver.getResults(res); + driver.run("drop table junit_unparted"); + Iterator itr = res.iterator(); + for (int i = 0; i < LOOP_SIZE * LOOP_SIZE; i++) { + Assert.assertEquals(input[i], itr.next()); + } + + Assert.assertFalse(itr.hasNext()); + + } + + @Test + public void testEmptyStore() throws IOException, CommandNeedRetryException { + + driver.run("drop table junit_unparted"); + String createTable = "create table junit_unparted(a int, b string) stored as RCFILE"; + int retCode = driver.run(createTable).getResponseCode(); + if (retCode != 0) { + throw new IOException("Failed to create table."); + } + + int LOOP_SIZE = 3; + String[] input = new String[LOOP_SIZE * LOOP_SIZE]; + int k = 0; + for (int i = 1; i <= LOOP_SIZE; i++) { + String si = i + ""; + for (int j = 1; j <= LOOP_SIZE; j++) { + input[k++] = si + "\t" + j; + } + } + HcatTestUtils.createTestDataFile(INPUT_FILE_NAME, input); + PigServer server = new PigServer(ExecType.LOCAL); + server.setBatchOn(); + server.registerQuery("A = load '" + INPUT_FILE_NAME + "' as (a:int, b:chararray);"); + server.registerQuery("B = filter A by a > 100;"); + server.registerQuery("store B into 'default.junit_unparted' using " + HCatStorer.class.getName() + "('','a:int,b:chararray');"); + server.executeBatch(); + + driver.run("select * from junit_unparted"); + ArrayList res = new ArrayList(); + driver.getResults(res); + driver.run("drop table junit_unparted"); + Iterator itr = res.iterator(); + Assert.assertFalse(itr.hasNext()); + + } + + @Test + public void testBagNStruct() throws IOException, CommandNeedRetryException { + driver.run("drop table junit_unparted"); + String createTable = "create table junit_unparted(b string,a struct, arr_of_struct array, " + + "arr_of_struct2 array>, arr_of_struct3 array>) stored as RCFILE"; + int retCode = driver.run(createTable).getResponseCode(); + if (retCode != 0) { + throw new IOException("Failed to create table."); + } + + String[] inputData = new String[]{"zookeeper\t(2)\t{(pig)}\t{(pnuts,hdfs)}\t{(hadoop),(hcat)}", + "chubby\t(2)\t{(sawzall)}\t{(bigtable,gfs)}\t{(mapreduce),(hcat)}"}; + + HcatTestUtils.createTestDataFile(INPUT_FILE_NAME, inputData); + + PigServer server = new PigServer(ExecType.LOCAL); + server.setBatchOn(); + server.registerQuery("A = load '" + INPUT_FILE_NAME + "' as (b:chararray, a:tuple(a1:int), arr_of_struct:bag{mytup:tuple(s1:chararray)}, arr_of_struct2:bag{mytup:tuple(s1:chararray,s2:chararray)}, arr_of_struct3:bag{t3:tuple(s3:chararray)});"); + server.registerQuery("store A into 'default.junit_unparted' using " + HCatStorer.class.getName() + "('','b:chararray, a:tuple(a1:int)," + + " arr_of_struct:bag{mytup:tuple(s1:chararray)}, arr_of_struct2:bag{mytup:tuple(s1:chararray,s2:chararray)}, arr_of_struct3:bag{t3:tuple(s3:chararray)}');"); + server.executeBatch(); + + driver.run("select * from junit_unparted"); + ArrayList res = new ArrayList(); + driver.getResults(res); + driver.run("drop table junit_unparted"); + Iterator itr = res.iterator(); + Assert.assertEquals("zookeeper\t{\"a1\":2}\t[\"pig\"]\t[{\"s1\":\"pnuts\",\"s2\":\"hdfs\"}]\t[{\"s3\":\"hadoop\"},{\"s3\":\"hcat\"}]", itr.next()); + Assert.assertEquals("chubby\t{\"a1\":2}\t[\"sawzall\"]\t[{\"s1\":\"bigtable\",\"s2\":\"gfs\"}]\t[{\"s3\":\"mapreduce\"},{\"s3\":\"hcat\"}]", itr.next()); + Assert.assertFalse(itr.hasNext()); + + } + + @Test + public void testStoreFuncAllSimpleTypes() throws IOException, CommandNeedRetryException { + + driver.run("drop table junit_unparted"); + String createTable = "create table junit_unparted(a int, b float, c double, d bigint, e string, f binary, g binary) stored as RCFILE"; + int retCode = driver.run(createTable).getResponseCode(); + if (retCode != 0) { + throw new IOException("Failed to create table."); + } + + int i = 0; + String[] input = new String[3]; + input[i++] = "0\t\t\t\t\t\t"; //Empty values except first column + input[i++] = "\t" + i * 2.1f + "\t" + i * 1.1d + "\t" + i * 2L + "\t" + "lets hcat" + "\tbinary-data"; //First column empty + input[i++] = i + "\t" + i * 2.1f + "\t" + i * 1.1d + "\t" + i * 2L + "\t" + "lets hcat" + "\tbinary-data"; + + HcatTestUtils.createTestDataFile(INPUT_FILE_NAME, input); + PigServer server = new PigServer(ExecType.LOCAL); + server.setBatchOn(); + server.registerQuery("A = load '" + INPUT_FILE_NAME + "' as (a:int, b:float, c:double, d:long, e:chararray, f:bytearray);"); + //null gets stored into column g which is a binary field. + server.registerQuery("store A into 'default.junit_unparted' using " + HCatStorer.class.getName() + "('','a:int, b:float, c:double, d:long, e:chararray,f:bytearray');"); + server.executeBatch(); + + + driver.run("select * from junit_unparted"); + ArrayList res = new ArrayList(); + driver.getResults(res); + + Iterator itr = res.iterator(); + Assert.assertEquals("0\tNULL\tNULL\tNULL\tNULL\tNULL\tNULL", itr.next()); + Assert.assertEquals("NULL\t4.2\t2.2\t4\tlets hcat\tbinary-data\tNULL", itr.next()); + Assert.assertEquals("3\t6.2999997\t3.3000000000000003\t6\tlets hcat\tbinary-data\tNULL", itr.next()); + Assert.assertFalse(itr.hasNext()); + + server.registerQuery("B = load 'junit_unparted' using " + HCatLoader.class.getName() + ";"); + Iterator iter = server.openIterator("B"); + int count = 0; + int num5nulls = 0; + while (iter.hasNext()) { + Tuple t = iter.next(); + if (t.get(5) == null) { + num5nulls++; + } else { + Assert.assertTrue(t.get(5) instanceof DataByteArray); + } + Assert.assertNull(t.get(6)); + count++; + } + Assert.assertEquals(3, count); + Assert.assertEquals(1, num5nulls); + driver.run("drop table junit_unparted"); + } + + @Test + public void testStoreFuncSimple() throws IOException, CommandNeedRetryException { + + driver.run("drop table junit_unparted"); + String createTable = "create table junit_unparted(a int, b string) stored as RCFILE"; + int retCode = driver.run(createTable).getResponseCode(); + if (retCode != 0) { + throw new IOException("Failed to create table."); + } + + int LOOP_SIZE = 3; + String[] inputData = new String[LOOP_SIZE * LOOP_SIZE]; + int k = 0; + for (int i = 1; i <= LOOP_SIZE; i++) { + String si = i + ""; + for (int j = 1; j <= LOOP_SIZE; j++) { + inputData[k++] = si + "\t" + j; + } + } + HcatTestUtils.createTestDataFile(INPUT_FILE_NAME, inputData); + PigServer server = new PigServer(ExecType.LOCAL); + server.setBatchOn(); + server.registerQuery("A = load '" + INPUT_FILE_NAME + "' as (a:int, b:chararray);"); + server.registerQuery("store A into 'default.junit_unparted' using " + HCatStorer.class.getName() + "('','a:int,b:chararray');"); + server.executeBatch(); + + driver.run("select * from junit_unparted"); + ArrayList res = new ArrayList(); + driver.getResults(res); + driver.run("drop table junit_unparted"); + Iterator itr = res.iterator(); + for (int i = 1; i <= LOOP_SIZE; i++) { + String si = i + ""; + for (int j = 1; j <= LOOP_SIZE; j++) { + Assert.assertEquals(si + "\t" + j, itr.next()); + } + } + Assert.assertFalse(itr.hasNext()); + + } + + @Test + public void testDynamicPartitioningMultiPartColsInDataPartialSpec() throws IOException, CommandNeedRetryException { + + driver.run("drop table if exists employee"); + String createTable = "CREATE TABLE employee (emp_id INT, emp_name STRING, emp_start_date STRING , emp_gender STRING ) " + + " PARTITIONED BY (emp_country STRING , emp_state STRING ) STORED AS RCFILE"; + + int retCode = driver.run(createTable).getResponseCode(); + if (retCode != 0) { + throw new IOException("Failed to create table."); + } + + String[] inputData = {"111237\tKrishna\t01/01/1990\tM\tIN\tTN", + "111238\tKalpana\t01/01/2000\tF\tIN\tKA", + "111239\tSatya\t01/01/2001\tM\tIN\tKL", + "111240\tKavya\t01/01/2002\tF\tIN\tAP"}; + + HcatTestUtils.createTestDataFile(INPUT_FILE_NAME, inputData); + PigServer pig = new PigServer(ExecType.LOCAL); + pig.setBatchOn(); + pig.registerQuery("A = LOAD '" + INPUT_FILE_NAME + "' USING PigStorage() AS (emp_id:int,emp_name:chararray,emp_start_date:chararray," + + "emp_gender:chararray,emp_country:chararray,emp_state:chararray);"); + pig.registerQuery("IN = FILTER A BY emp_country == 'IN';"); + pig.registerQuery("STORE IN INTO 'employee' USING " + HCatStorer.class.getName() + "('emp_country=IN');"); + pig.executeBatch(); + driver.run("select * from employee"); + ArrayList results = new ArrayList(); + driver.getResults(results); + Assert.assertEquals(4, results.size()); + Collections.sort(results); + Assert.assertEquals(inputData[0], results.get(0)); + Assert.assertEquals(inputData[1], results.get(1)); + Assert.assertEquals(inputData[2], results.get(2)); + Assert.assertEquals(inputData[3], results.get(3)); + driver.run("drop table employee"); + } + + @Test + public void testDynamicPartitioningMultiPartColsInDataNoSpec() throws IOException, CommandNeedRetryException { + + driver.run("drop table if exists employee"); + String createTable = "CREATE TABLE employee (emp_id INT, emp_name STRING, emp_start_date STRING , emp_gender STRING ) " + + " PARTITIONED BY (emp_country STRING , emp_state STRING ) STORED AS RCFILE"; + + int retCode = driver.run(createTable).getResponseCode(); + if (retCode != 0) { + throw new IOException("Failed to create table."); + } + + String[] inputData = {"111237\tKrishna\t01/01/1990\tM\tIN\tTN", + "111238\tKalpana\t01/01/2000\tF\tIN\tKA", + "111239\tSatya\t01/01/2001\tM\tIN\tKL", + "111240\tKavya\t01/01/2002\tF\tIN\tAP"}; + + HcatTestUtils.createTestDataFile(INPUT_FILE_NAME, inputData); + PigServer pig = new PigServer(ExecType.LOCAL); + pig.setBatchOn(); + pig.registerQuery("A = LOAD '" + INPUT_FILE_NAME + "' USING PigStorage() AS (emp_id:int,emp_name:chararray,emp_start_date:chararray," + + "emp_gender:chararray,emp_country:chararray,emp_state:chararray);"); + pig.registerQuery("IN = FILTER A BY emp_country == 'IN';"); + pig.registerQuery("STORE IN INTO 'employee' USING " + HCatStorer.class.getName() + "();"); + pig.executeBatch(); + driver.run("select * from employee"); + ArrayList results = new ArrayList(); + driver.getResults(results); + Assert.assertEquals(4, results.size()); + Collections.sort(results); + Assert.assertEquals(inputData[0], results.get(0)); + Assert.assertEquals(inputData[1], results.get(1)); + Assert.assertEquals(inputData[2], results.get(2)); + Assert.assertEquals(inputData[3], results.get(3)); + driver.run("drop table employee"); + } + + @Test + public void testDynamicPartitioningMultiPartColsNoDataInDataNoSpec() throws IOException, CommandNeedRetryException { + + driver.run("drop table if exists employee"); + String createTable = "CREATE TABLE employee (emp_id INT, emp_name STRING, emp_start_date STRING , emp_gender STRING ) " + + " PARTITIONED BY (emp_country STRING , emp_state STRING ) STORED AS RCFILE"; + + int retCode = driver.run(createTable).getResponseCode(); + if (retCode != 0) { + throw new IOException("Failed to create table."); + } + + String[] inputData = {}; + HcatTestUtils.createTestDataFile(INPUT_FILE_NAME, inputData); + + PigServer pig = new PigServer(ExecType.LOCAL); + pig.setBatchOn(); + pig.registerQuery("A = LOAD '" + INPUT_FILE_NAME + "' USING PigStorage() AS (emp_id:int,emp_name:chararray,emp_start_date:chararray," + + "emp_gender:chararray,emp_country:chararray,emp_state:chararray);"); + pig.registerQuery("IN = FILTER A BY emp_country == 'IN';"); + pig.registerQuery("STORE IN INTO 'employee' USING " + HCatStorer.class.getName() + "();"); + pig.executeBatch(); + driver.run("select * from employee"); + ArrayList results = new ArrayList(); + driver.getResults(results); + Assert.assertEquals(0, results.size()); + driver.run("drop table employee"); + } + + public void testPartitionPublish() + throws IOException, CommandNeedRetryException { + + driver.run("drop table ptn_fail"); + String createTable = "create table ptn_fail(a int, c string) partitioned by (b string) stored as RCFILE"; + int retCode = driver.run(createTable).getResponseCode(); + if (retCode != 0) { + throw new IOException("Failed to create table."); + } + int LOOP_SIZE = 11; + String[] input = new String[LOOP_SIZE]; + + for (int i = 0; i < LOOP_SIZE; i++) { + input[i] = i + "\tmath"; + } + HcatTestUtils.createTestDataFile(INPUT_FILE_NAME, input); + PigServer server = new PigServer(ExecType.LOCAL); + server.setBatchOn(); + server.registerQuery("A = load '" + INPUT_FILE_NAME + + "' as (a:int, c:chararray);"); + server.registerQuery("B = filter A by " + FailEvalFunc.class.getName() + + "($0);"); + server.registerQuery("store B into 'ptn_fail' using " + + HCatStorer.class.getName() + "('b=math');"); + server.executeBatch(); + + String query = "show partitions ptn_fail"; + retCode = driver.run(query).getResponseCode(); + + if (retCode != 0) { + throw new IOException("Error " + retCode + " running query " + + query); + } + + ArrayList res = new ArrayList(); + driver.getResults(res); + Assert.assertEquals(0, res.size()); + + // Make sure the partitions directory is not in hdfs. + Assert.assertTrue((new File(TEST_WAREHOUSE_DIR + "/ptn_fail")).exists()); + Assert.assertFalse((new File(TEST_WAREHOUSE_DIR + "/ptn_fail/b=math")) + .exists()); + } + + static public class FailEvalFunc extends EvalFunc { + + /* + * @param Tuple /* @return null /* @throws IOException + * + * @see org.apache.pig.EvalFunc#exec(org.apache.pig.data.Tuple) + */ + @Override + public Boolean exec(Tuple tuple) throws IOException { + throw new IOException("Eval Func to mimic Failure."); + } + + } +} diff --git hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/TestHCatStorerMulti.java hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/TestHCatStorerMulti.java new file mode 100644 index 0000000..a90e9c3 --- /dev/null +++ hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/TestHCatStorerMulti.java @@ -0,0 +1,204 @@ +/** + * 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.hcatalog.pig; + +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Map; + +import junit.framework.TestCase; + +import org.apache.hadoop.fs.FileUtil; +import org.apache.hadoop.hive.cli.CliSessionState; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.CommandNeedRetryException; +import org.apache.hadoop.hive.ql.Driver; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hcatalog.data.Pair; +import org.apache.pig.ExecType; +import org.apache.pig.PigServer; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.pig.TestHCatStorerMulti} instead + */ +public class TestHCatStorerMulti extends TestCase { + private static final String TEST_DATA_DIR = System.getProperty("user.dir") + + "/build/test/data/" + TestHCatStorerMulti.class.getCanonicalName(); + private static final String TEST_WAREHOUSE_DIR = TEST_DATA_DIR + "/warehouse"; + private static final String INPUT_FILE_NAME = TEST_DATA_DIR + "/input.data"; + + private static final String BASIC_TABLE = "junit_unparted_basic"; + private static final String PARTITIONED_TABLE = "junit_parted_basic"; + private static Driver driver; + + private static Map> basicInputData; + + protected String storageFormat() { + return "RCFILE tblproperties('hcat.isd'='org.apache.hcatalog.rcfile.RCFileInputDriver'," + + "'hcat.osd'='org.apache.hcatalog.rcfile.RCFileOutputDriver')"; + } + + private void dropTable(String tablename) throws IOException, CommandNeedRetryException { + driver.run("drop table " + tablename); + } + + private void createTable(String tablename, String schema, String partitionedBy) throws IOException, CommandNeedRetryException { + String createTable; + createTable = "create table " + tablename + "(" + schema + ") "; + if ((partitionedBy != null) && (!partitionedBy.trim().isEmpty())) { + createTable = createTable + "partitioned by (" + partitionedBy + ") "; + } + createTable = createTable + "stored as " + storageFormat(); + int retCode = driver.run(createTable).getResponseCode(); + if (retCode != 0) { + throw new IOException("Failed to create table. [" + createTable + "], return code from hive driver : [" + retCode + "]"); + } + } + + private void createTable(String tablename, String schema) throws IOException, CommandNeedRetryException { + createTable(tablename, schema, null); + } + + @Override + protected void setUp() throws Exception { + if (driver == null) { + HiveConf hiveConf = new HiveConf(this.getClass()); + hiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); + hiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); + hiveConf.set(HiveConf.ConfVars.METASTOREWAREHOUSE.varname, TEST_WAREHOUSE_DIR); + driver = new Driver(hiveConf); + SessionState.start(new CliSessionState(hiveConf)); + } + + cleanup(); + } + + @Override + protected void tearDown() throws Exception { + cleanup(); + } + + public void testStoreBasicTable() throws Exception { + + + createTable(BASIC_TABLE, "a int, b string"); + + populateBasicFile(); + + PigServer server = new PigServer(ExecType.LOCAL); + server.setBatchOn(); + server.registerQuery("A = load '" + INPUT_FILE_NAME + "' as (a:int, b:chararray);"); + server.registerQuery("store A into '" + BASIC_TABLE + "' using org.apache.hcatalog.pig.HCatStorer();"); + + server.executeBatch(); + + driver.run("select * from " + BASIC_TABLE); + ArrayList unpartitionedTableValuesReadFromHiveDriver = new ArrayList(); + driver.getResults(unpartitionedTableValuesReadFromHiveDriver); + assertEquals(basicInputData.size(), unpartitionedTableValuesReadFromHiveDriver.size()); + } + + public void testStorePartitionedTable() throws Exception { + createTable(PARTITIONED_TABLE, "a int, b string", "bkt string"); + + populateBasicFile(); + + PigServer server = new PigServer(ExecType.LOCAL); + server.setBatchOn(); + server.registerQuery("A = load '" + INPUT_FILE_NAME + "' as (a:int, b:chararray);"); + + server.registerQuery("B2 = filter A by a < 2;"); + server.registerQuery("store B2 into '" + PARTITIONED_TABLE + "' using org.apache.hcatalog.pig.HCatStorer('bkt=0');"); + server.registerQuery("C2 = filter A by a >= 2;"); + server.registerQuery("store C2 into '" + PARTITIONED_TABLE + "' using org.apache.hcatalog.pig.HCatStorer('bkt=1');"); + + server.executeBatch(); + + driver.run("select * from " + PARTITIONED_TABLE); + ArrayList partitionedTableValuesReadFromHiveDriver = new ArrayList(); + driver.getResults(partitionedTableValuesReadFromHiveDriver); + assertEquals(basicInputData.size(), partitionedTableValuesReadFromHiveDriver.size()); + } + + public void testStoreTableMulti() throws Exception { + + + createTable(BASIC_TABLE, "a int, b string"); + createTable(PARTITIONED_TABLE, "a int, b string", "bkt string"); + + populateBasicFile(); + + PigServer server = new PigServer(ExecType.LOCAL); + server.setBatchOn(); + server.registerQuery("A = load '" + INPUT_FILE_NAME + "' as (a:int, b:chararray);"); + server.registerQuery("store A into '" + BASIC_TABLE + "' using org.apache.hcatalog.pig.HCatStorer();"); + + server.registerQuery("B2 = filter A by a < 2;"); + server.registerQuery("store B2 into '" + PARTITIONED_TABLE + "' using org.apache.hcatalog.pig.HCatStorer('bkt=0');"); + server.registerQuery("C2 = filter A by a >= 2;"); + server.registerQuery("store C2 into '" + PARTITIONED_TABLE + "' using org.apache.hcatalog.pig.HCatStorer('bkt=1');"); + + server.executeBatch(); + + driver.run("select * from " + BASIC_TABLE); + ArrayList unpartitionedTableValuesReadFromHiveDriver = new ArrayList(); + driver.getResults(unpartitionedTableValuesReadFromHiveDriver); + driver.run("select * from " + PARTITIONED_TABLE); + ArrayList partitionedTableValuesReadFromHiveDriver = new ArrayList(); + driver.getResults(partitionedTableValuesReadFromHiveDriver); + assertEquals(basicInputData.size(), unpartitionedTableValuesReadFromHiveDriver.size()); + assertEquals(basicInputData.size(), partitionedTableValuesReadFromHiveDriver.size()); + } + + private void populateBasicFile() throws IOException { + int LOOP_SIZE = 3; + String[] input = new String[LOOP_SIZE * LOOP_SIZE]; + basicInputData = new HashMap>(); + int k = 0; + File file = new File(INPUT_FILE_NAME); + file.deleteOnExit(); + FileWriter writer = new FileWriter(file); + for (int i = 1; i <= LOOP_SIZE; i++) { + String si = i + ""; + for (int j = 1; j <= LOOP_SIZE; j++) { + String sj = "S" + j + "S"; + input[k] = si + "\t" + sj; + basicInputData.put(k, new Pair(i, sj)); + writer.write(input[k] + "\n"); + k++; + } + } + writer.close(); + } + + private void cleanup() throws IOException, CommandNeedRetryException { + File f = new File(TEST_WAREHOUSE_DIR); + if (f.exists()) { + FileUtil.fullyDelete(f); + } + new File(TEST_WAREHOUSE_DIR).mkdirs(); + + dropTable(BASIC_TABLE); + dropTable(PARTITIONED_TABLE); + } +} diff --git hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/TestHCatStorerWrapper.java hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/TestHCatStorerWrapper.java new file mode 100644 index 0000000..755d437 --- /dev/null +++ hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/TestHCatStorerWrapper.java @@ -0,0 +1,94 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.hcatalog.pig; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.UUID; + +import org.apache.commons.lang.SystemUtils; +import org.apache.hadoop.hive.ql.CommandNeedRetryException; +import org.apache.hcatalog.HcatTestUtils; +import org.apache.hcatalog.mapreduce.HCatBaseTest; +import org.apache.pig.ExecType; +import org.apache.pig.PigServer; +import org.junit.Assert; +import org.junit.Test; + +/** + * This test checks the {@link HCatConstants#HCAT_PIG_STORER_EXTERNAL_LOCATION} that we can set in the + * UDFContext of {@link HCatStorer} so that it writes to the specified external location. + * + * Since {@link HCatStorer} does not allow extra parameters in the constructor, we use {@link HCatStorerWrapper} + * that always treats the last parameter as the external path. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.pig.TestHCatStorerWrapper} instead + */ +public class TestHCatStorerWrapper extends HCatBaseTest { + + private static final String INPUT_FILE_NAME = TEST_DATA_DIR + "/input.data"; + + @Test + public void testStoreExternalTableWithExternalDir() throws IOException, CommandNeedRetryException{ + + File tmpExternalDir = new File(SystemUtils.getJavaIoTmpDir(), UUID.randomUUID().toString()); + tmpExternalDir.deleteOnExit(); + + String part_val = "100"; + + driver.run("drop table junit_external"); + String createTable = "create external table junit_external(a int, b string) partitioned by (c string) stored as RCFILE"; + Assert.assertEquals(0, driver.run(createTable).getResponseCode()); + + int LOOP_SIZE = 3; + String[] inputData = new String[LOOP_SIZE*LOOP_SIZE]; + int k = 0; + for(int i = 1; i <= LOOP_SIZE; i++) { + String si = i + ""; + for(int j=1;j<=LOOP_SIZE;j++) { + inputData[k++] = si + "\t"+j; + } + } + HcatTestUtils.createTestDataFile(INPUT_FILE_NAME, inputData); + PigServer server = new PigServer(ExecType.LOCAL); + server.setBatchOn(); + logAndRegister(server, "A = load '"+INPUT_FILE_NAME+"' as (a:int, b:chararray);"); + logAndRegister(server, "store A into 'default.junit_external' using " + HCatStorerWrapper.class.getName() + + "('c=" + part_val + "','" + tmpExternalDir.getAbsolutePath() + "');"); + server.executeBatch(); + + Assert.assertTrue(tmpExternalDir.exists()); + Assert.assertTrue(new File(tmpExternalDir.getAbsoluteFile() + "/" + "part-m-00000").exists()); + + driver.run("select * from junit_external"); + ArrayList res = new ArrayList(); + driver.getResults(res); + driver.run("drop table junit_external"); + Iterator itr = res.iterator(); + for(int i = 1; i <= LOOP_SIZE; i++) { + String si = i + ""; + for(int j=1;j<=LOOP_SIZE;j++) { + Assert.assertEquals( si + "\t" + j + "\t" + part_val,itr.next()); + } + } + Assert.assertFalse(itr.hasNext()); + + } +} diff --git hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/TestOrcHCatLoader.java hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/TestOrcHCatLoader.java new file mode 100644 index 0000000..7f3e5d3 --- /dev/null +++ hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/TestOrcHCatLoader.java @@ -0,0 +1,32 @@ +/** + * 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.hcatalog.pig; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.pig.TestOrcHCatLoader} instead + */ +public class TestOrcHCatLoader extends TestHCatLoader { + + @Override + protected String storageFormat() { + return "orc"; + } + +} + diff --git hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/TestOrcHCatStorer.java hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/TestOrcHCatStorer.java new file mode 100644 index 0000000..4afbaa0 --- /dev/null +++ hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/TestOrcHCatStorer.java @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.hcatalog.pig; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.pig.TestOrcHCatStorer} instead + */ +public class TestOrcHCatStorer extends TestHCatStorerMulti { + + @Override + protected String storageFormat() { + return "orc"; + } +} + diff --git hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/TestPermsInheritance.java.broken hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/TestPermsInheritance.java.broken new file mode 100644 index 0000000..fce1e70 --- /dev/null +++ hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/TestPermsInheritance.java.broken @@ -0,0 +1,135 @@ +/** + * 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. + */ + +-->There are two pieces of code that sets directory permissions. +-->One that sets the UMask which only woks for dfs filesystem. +-->And the other change the permission of directories after they are created. +-->I removed that since it is not secure and just add more load on the namenode. +-->We should push this test to e2e to verify what actually runs in production. + +package org.apache.hcatalog.pig; + +import java.io.IOException; + +import junit.framework.TestCase; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.MetaStoreUtils; +import org.apache.hadoop.hive.metastore.Warehouse; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hadoop.hive.metastore.api.UnknownTableException; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hcatalog.ExitException; +import org.apache.hcatalog.NoExitSecurityManager; +import org.apache.hcatalog.cli.HCatCli; +import org.apache.hcatalog.pig.HCatStorer; +import org.apache.pig.ExecType; +import org.apache.pig.PigServer; +import org.apache.pig.impl.util.UDFContext; +import org.apache.thrift.TException; + +public class TestPermsInheritance extends TestCase { + + @Override + protected void setUp() throws Exception { + super.setUp(); + securityManager = System.getSecurityManager(); + System.setSecurityManager(new NoExitSecurityManager()); + msc = new HiveMetaStoreClient(conf); + msc.dropTable(MetaStoreUtils.DEFAULT_DATABASE_NAME,"testNoPartTbl", true,true); + System.setProperty(HiveConf.ConfVars.PREEXECHOOKS.varname, " "); + System.setProperty(HiveConf.ConfVars.POSTEXECHOOKS.varname, " "); + msc.dropTable(MetaStoreUtils.DEFAULT_DATABASE_NAME,"testPartTbl", true,true); + pig = new PigServer(ExecType.LOCAL, conf.getAllProperties()); + UDFContext.getUDFContext().setClientSystemProps(); + } + + private HiveMetaStoreClient msc; + private SecurityManager securityManager; + private PigServer pig; + + @Override + protected void tearDown() throws Exception { + super.tearDown(); + System.setSecurityManager(securityManager); + } + + private final HiveConf conf = new HiveConf(this.getClass()); + + public void testNoPartTbl() throws IOException, MetaException, UnknownTableException, TException, NoSuchObjectException, HiveException{ + + try{ + HCatCli.main(new String[]{"-e","create table testNoPartTbl (line string) stored as RCFILE", "-p","rwx-wx---"}); + } + catch(Exception e){ + assertTrue(e instanceof ExitException); + assertEquals(((ExitException)e).getStatus(), 0); + } + Warehouse wh = new Warehouse(conf); + Path dfsPath = wh.getTablePath(Hive.get(conf).getDatabase(MetaStoreUtils.DEFAULT_DATABASE_NAME), "testNoPartTbl"); + FileSystem fs = dfsPath.getFileSystem(conf); + assertEquals(fs.getFileStatus(dfsPath).getPermission(),FsPermission.valueOf("drwx-wx---")); + + pig.setBatchOn(); + pig.registerQuery("A = load 'build.xml' as (line:chararray);"); + pig.registerQuery("store A into 'testNoPartTbl' using "+HCatStorer.class.getName()+"();"); + pig.executeBatch(); + FileStatus[] status = fs.listStatus(dfsPath,hiddenFileFilter); + + assertEquals(status.length, 1); + assertEquals(FsPermission.valueOf("drwx-wx---"),status[0].getPermission()); + + try{ + HCatCli.main(new String[]{"-e","create table testPartTbl (line string) partitioned by (a string) stored as RCFILE", "-p","rwx-wx--x"}); + } + catch(Exception e){ + assertTrue(e instanceof ExitException); + assertEquals(((ExitException)e).getStatus(), 0); + } + + dfsPath = wh.getTablePath(Hive.get(conf).getDatabase(MetaStoreUtils.DEFAULT_DATABASE_NAME), "testPartTbl"); + assertEquals(fs.getFileStatus(dfsPath).getPermission(),FsPermission.valueOf("drwx-wx--x")); + + pig.setBatchOn(); + pig.registerQuery("A = load 'build.xml' as (line:chararray);"); + pig.registerQuery("store A into 'testPartTbl' using "+HCatStorer.class.getName()+"('a=part');"); + pig.executeBatch(); + + Path partPath = new Path(dfsPath,"a=part"); + assertEquals(FsPermission.valueOf("drwx-wx--x"),fs.getFileStatus(partPath).getPermission()); + status = fs.listStatus(partPath,hiddenFileFilter); + assertEquals(status.length, 1); + assertEquals(FsPermission.valueOf("drwx-wx--x"),status[0].getPermission()); + } + + private static final PathFilter hiddenFileFilter = new PathFilter(){ + public boolean accept(Path p){ + String name = p.getName(); + return !name.startsWith("_") && !name.startsWith("."); + } + }; +} diff --git hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/TestPigHCatUtil.java hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/TestPigHCatUtil.java new file mode 100644 index 0000000..7ba7179 --- /dev/null +++ hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/TestPigHCatUtil.java @@ -0,0 +1,94 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hcatalog.pig; + +import com.google.common.collect.Lists; +import junit.framework.Assert; +import org.apache.hcatalog.common.HCatConstants; +import org.apache.hcatalog.data.schema.HCatFieldSchema; +import org.apache.hcatalog.data.schema.HCatSchema; +import org.apache.pig.ResourceSchema; +import org.apache.pig.ResourceSchema.ResourceFieldSchema; +import org.apache.pig.data.DataType; +import org.apache.pig.impl.util.UDFContext; +import org.junit.Test; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.pig.TestPigHCatUtil} instead + */ +public class TestPigHCatUtil { + + @Test + public void testGetBagSubSchema() throws Exception { + + // Define the expected schema. + ResourceFieldSchema[] bagSubFieldSchemas = new ResourceFieldSchema[1]; + bagSubFieldSchemas[0] = new ResourceFieldSchema().setName("innertuple") + .setDescription("The tuple in the bag").setType(DataType.TUPLE); + + ResourceFieldSchema[] innerTupleFieldSchemas = new ResourceFieldSchema[1]; + innerTupleFieldSchemas[0] = + new ResourceFieldSchema().setName("innerfield").setType(DataType.CHARARRAY); + + bagSubFieldSchemas[0].setSchema(new ResourceSchema().setFields(innerTupleFieldSchemas)); + ResourceSchema expected = new ResourceSchema().setFields(bagSubFieldSchemas); + + // Get the actual converted schema. + HCatSchema hCatSchema = new HCatSchema(Lists.newArrayList( + new HCatFieldSchema("innerLlama", HCatFieldSchema.Type.STRING, null))); + HCatFieldSchema hCatFieldSchema = + new HCatFieldSchema("llama", HCatFieldSchema.Type.ARRAY, hCatSchema, null); + ResourceSchema actual = PigHCatUtil.getBagSubSchema(hCatFieldSchema); + + Assert.assertEquals(expected.toString(), actual.toString()); + } + + @Test + public void testGetBagSubSchemaConfigured() throws Exception { + + // NOTE: pig-0.8 sets client system properties by actually getting the client + // system properties. Starting in pig-0.9 you must pass the properties in. + // When updating our pig dependency this will need updated. + System.setProperty(HCatConstants.HCAT_PIG_INNER_TUPLE_NAME, "t"); + System.setProperty(HCatConstants.HCAT_PIG_INNER_FIELD_NAME, "FIELDNAME_tuple"); + UDFContext.getUDFContext().setClientSystemProps(System.getProperties()); + + // Define the expected schema. + ResourceFieldSchema[] bagSubFieldSchemas = new ResourceFieldSchema[1]; + bagSubFieldSchemas[0] = new ResourceFieldSchema().setName("t") + .setDescription("The tuple in the bag").setType(DataType.TUPLE); + + ResourceFieldSchema[] innerTupleFieldSchemas = new ResourceFieldSchema[1]; + innerTupleFieldSchemas[0] = + new ResourceFieldSchema().setName("llama_tuple").setType(DataType.CHARARRAY); + + bagSubFieldSchemas[0].setSchema(new ResourceSchema().setFields(innerTupleFieldSchemas)); + ResourceSchema expected = new ResourceSchema().setFields(bagSubFieldSchemas); + + // Get the actual converted schema. + HCatSchema actualHCatSchema = new HCatSchema(Lists.newArrayList( + new HCatFieldSchema("innerLlama", HCatFieldSchema.Type.STRING, null))); + HCatFieldSchema actualHCatFieldSchema = + new HCatFieldSchema("llama", HCatFieldSchema.Type.ARRAY, actualHCatSchema, null); + ResourceSchema actual = PigHCatUtil.getBagSubSchema(actualHCatFieldSchema); + + Assert.assertEquals(expected.toString(), actual.toString()); + } +} diff --git hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/TestPigStorageDriver.java.broken hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/TestPigStorageDriver.java.broken new file mode 100644 index 0000000..fdf3a98 --- /dev/null +++ hcatalog/hcatalog-pig-adapter/src/test/java/org/apache/hcatalog/pig/TestPigStorageDriver.java.broken @@ -0,0 +1,272 @@ +/** + * 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.hcatalog.pig; + +import java.io.BufferedInputStream; +import java.io.DataInputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileWriter; +import java.io.IOException; +import java.io.PrintWriter; +import java.util.Iterator; +import java.util.Map; + +import junit.framework.TestCase; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.cli.CliSessionState; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.MetaStoreUtils; +import org.apache.hadoop.hive.metastore.api.InvalidOperationException; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.UnknownTableException; +import org.apache.hadoop.hive.ql.CommandNeedRetryException; +import org.apache.hadoop.hive.ql.Driver; +import org.apache.hadoop.hive.ql.io.RCFileInputFormat; +import org.apache.hadoop.hive.ql.io.RCFileOutputFormat; +import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hcatalog.cli.SemanticAnalysis.HCatSemanticAnalyzer; +import org.apache.hcatalog.common.HCatConstants; +import org.apache.hcatalog.pig.HCatLoader; +import org.apache.pig.ExecType; +import org.apache.pig.PigServer; +import org.apache.pig.data.Tuple; +import org.apache.pig.impl.logicalLayer.FrontendException; +import org.apache.pig.impl.util.UDFContext; +import org.apache.thrift.TException; + +public class TestPigStorageDriver extends TestCase { + + private HiveConf hcatConf; + private Driver hcatDriver; + private HiveMetaStoreClient msc; + private static String tblLocation = "/tmp/test_pig/data"; + private static String anyExistingFileInCurDir = "ivy.xml"; + private static String warehouseDir = "/tmp/hcat_junit_warehouse"; + + @Override + protected void setUp() throws Exception { + + hcatConf = new HiveConf(this.getClass()); + hcatConf.set(ConfVars.PREEXECHOOKS.varname, ""); + hcatConf.set(ConfVars.POSTEXECHOOKS.varname, ""); + hcatConf.set(ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false"); + hcatConf.set(ConfVars.SEMANTIC_ANALYZER_HOOK.varname, HCatSemanticAnalyzer.class.getName()); + hcatDriver = new Driver(hcatConf); + msc = new HiveMetaStoreClient(hcatConf); + SessionState.start(new CliSessionState(hcatConf)); + super.setUp(); + } + + @Override + protected void tearDown() throws Exception { + super.tearDown(); + } + + public void testPigStorageDriver() throws IOException, CommandNeedRetryException{ + + String fsLoc = hcatConf.get("fs.default.name"); + Path tblPath = new Path(fsLoc, tblLocation); + String tblName = "junit_pigstorage"; + tblPath.getFileSystem(hcatConf).copyFromLocalFile(new Path(anyExistingFileInCurDir),tblPath); + + hcatDriver.run("drop table " + tblName); + CommandProcessorResponse resp; + String createTable = "create table " + tblName + " (a string) partitioned by (b string) stored as TEXTFILE"; + + resp = hcatDriver.run(createTable); + assertEquals(0, resp.getResponseCode()); + assertNull(resp.getErrorMessage()); + + resp = hcatDriver.run("alter table " + tblName + " add partition (b='2010-10-10') location '"+new Path(fsLoc, "/tmp/test_pig")+"'"); + assertEquals(0, resp.getResponseCode()); + assertNull(resp.getErrorMessage()); + + resp = hcatDriver.run("alter table " + tblName + " partition (b='2010-10-10') set fileformat TEXTFILE"); + assertEquals(0, resp.getResponseCode()); + assertNull(resp.getErrorMessage()); + + resp = hcatDriver.run("desc extended " + tblName + " partition (b='2010-10-10')"); + assertEquals(0, resp.getResponseCode()); + assertNull(resp.getErrorMessage()); + + PigServer server = new PigServer(ExecType.LOCAL, hcatConf.getAllProperties()); + UDFContext.getUDFContext().setClientSystemProps(); + server.registerQuery(" a = load '" + tblName + "' using "+HCatLoader.class.getName()+";"); + Iterator itr = server.openIterator("a"); + boolean result = compareWithFile(itr, anyExistingFileInCurDir, 2, "2010-10-10", null); + assertTrue(result); + + server.registerQuery("a = load '"+tblPath.toString()+"' using PigStorage() as (a:chararray);"); + server.store("a", tblName, HCatStorer.class.getName() + "('b=2010-10-11')"); + + server.registerQuery("a = load '" + warehouseDir + "/" + tblName + "/b=2010-10-11' using PigStorage() as (a:chararray);"); + itr = server.openIterator("a"); + result = compareWithFile(itr, anyExistingFileInCurDir, 1, "2010-10-11", null); + assertTrue(result); + + // Test multi-store + server.registerQuery("a = load '"+tblPath.toString()+"' using PigStorage() as (a:chararray);"); + server.registerQuery("store a into '" + tblName + "' using " + HCatStorer.class.getName() + "('b=2010-11-01');"); + server.registerQuery("store a into '" + tblName + "' using " + HCatStorer.class.getName() + "('b=2010-11-02');"); + + server.registerQuery("a = load '" + warehouseDir + "/" + tblName + "/b=2010-11-01' using PigStorage() as (a:chararray);"); + itr = server.openIterator("a"); + result = compareWithFile(itr, anyExistingFileInCurDir, 1, "2010-11-01", null); + assertTrue(result); + + server.registerQuery("a = load '" + warehouseDir + "/" + tblName + "/b=2010-11-02' using PigStorage() as (a:chararray);"); + itr = server.openIterator("a"); + result = compareWithFile(itr, anyExistingFileInCurDir, 1, "2010-11-02", null); + assertTrue(result); + + hcatDriver.run("drop table " + tblName); + } + + private boolean compareWithFile(Iterator itr, String factFile, int numColumn, String key, String valueSuffix) throws IOException { + DataInputStream stream = new DataInputStream(new BufferedInputStream(new FileInputStream(new File(factFile)))); + while(itr.hasNext()){ + Tuple t = itr.next(); + assertEquals(numColumn, t.size()); + if(t.get(0) != null) { + // If underlying data-field is empty. PigStorage inserts null instead + // of empty String objects. + assertTrue(t.get(0) instanceof String); + String expected = stream.readLine(); + if (valueSuffix!=null) + expected += valueSuffix; + assertEquals(expected, t.get(0)); + } + else{ + assertTrue(stream.readLine().isEmpty()); + } + + if (numColumn>1) { + // The second column must be key + assertTrue(t.get(1) instanceof String); + assertEquals(key, t.get(1)); + } + } + assertEquals(0,stream.available()); + stream.close(); + return true; + } + + public void testDelim() throws MetaException, TException, UnknownTableException, NoSuchObjectException, InvalidOperationException, IOException, CommandNeedRetryException{ + + hcatDriver.run("drop table junit_pigstorage_delim"); + + CommandProcessorResponse resp; + String createTable = "create table junit_pigstorage_delim (a0 string, a1 string) partitioned by (b string) stored as RCFILE"; + + resp = hcatDriver.run(createTable); + + assertEquals(0, resp.getResponseCode()); + assertNull(resp.getErrorMessage()); + + resp = hcatDriver.run("alter table junit_pigstorage_delim add partition (b='2010-10-10')"); + assertEquals(0, resp.getResponseCode()); + assertNull(resp.getErrorMessage()); + + resp = hcatDriver.run("alter table junit_pigstorage_delim partition (b='2010-10-10') set fileformat TEXTFILE"); + + Partition part = msc.getPartition(MetaStoreUtils.DEFAULT_DATABASE_NAME, "junit_pigstorage_delim", "b=2010-10-10"); + Map partParms = part.getParameters(); + partParms.put(HCatConstants.HCAT_PIG_LOADER_ARGS, "control-A"); + partParms.put(HCatConstants.HCAT_PIG_STORER_ARGS, "control-A"); + + msc.alter_partition(MetaStoreUtils.DEFAULT_DATABASE_NAME, "junit_pigstorage_delim", part); + + PigServer server = new PigServer(ExecType.LOCAL, hcatConf.getAllProperties()); + UDFContext.getUDFContext().setClientSystemProps(); + server.registerQuery(" a = load 'junit_pigstorage_delim' using "+HCatLoader.class.getName()+";"); + try{ + server.openIterator("a"); + }catch(FrontendException fe){} + + resp = hcatDriver.run("alter table junit_pigstorage_delim set fileformat TEXTFILE"); + assertEquals(0, resp.getResponseCode()); + assertNull(resp.getErrorMessage()); + resp = hcatDriver.run("alter table junit_pigstorage_delim set TBLPROPERTIES ('hcat.pig.loader.args'=':', 'hcat.pig.storer.args'=':')"); + assertEquals(0, resp.getResponseCode()); + assertNull(resp.getErrorMessage()); + + File inputFile = File.createTempFile("hcat_test", ""); + PrintWriter p = new PrintWriter(new FileWriter(inputFile)); + p.println("1\t2"); + p.println("3\t4"); + p.close(); + server.registerQuery("a = load '"+inputFile.toString()+"' using PigStorage() as (a0:chararray, a1:chararray);"); + server.store("a", "junit_pigstorage_delim", HCatStorer.class.getName() + "('b=2010-10-11')"); + + server.registerQuery("a = load '/tmp/hcat_junit_warehouse/junit_pigstorage_delim/b=2010-10-11' using PigStorage() as (a:chararray);"); + Iterator itr = server.openIterator("a"); + + assertTrue(itr.hasNext()); + Tuple t = itr.next(); + assertTrue(t.get(0).equals("1:2")); + + assertTrue(itr.hasNext()); + t = itr.next(); + assertTrue(t.get(0).equals("3:4")); + + assertFalse(itr.hasNext()); + inputFile.delete(); + } + + public void testMultiConstructArgs() throws MetaException, TException, UnknownTableException, NoSuchObjectException, InvalidOperationException, IOException, CommandNeedRetryException{ + + String fsLoc = hcatConf.get("fs.default.name"); + Path tblPath = new Path(fsLoc, tblLocation); + String tblName = "junit_pigstorage_constructs"; + tblPath.getFileSystem(hcatConf).copyFromLocalFile(new Path(anyExistingFileInCurDir),tblPath); + + hcatDriver.run("drop table junit_pigstorage_constructs"); + + CommandProcessorResponse resp; + String createTable = "create table " + tblName + " (a string) partitioned by (b string) stored as TEXTFILE"; + + resp = hcatDriver.run(createTable); + + assertEquals(0, resp.getResponseCode()); + assertNull(resp.getErrorMessage()); + + resp = hcatDriver.run("alter table " + tblName + " set TBLPROPERTIES ('hcat.pig.storer'='org.apache.hcatalog.pig.MyPigStorage', 'hcat.pig.storer.args'=':#hello', 'hcat.pig.args.delimiter'='#')"); + assertEquals(0, resp.getResponseCode()); + assertNull(resp.getErrorMessage()); + + PigServer server = new PigServer(ExecType.LOCAL, hcatConf.getAllProperties()); + UDFContext.getUDFContext().setClientSystemProps(); + + server.registerQuery("a = load '"+tblPath.toString()+"' using PigStorage() as (a:chararray);"); + server.store("a", tblName, HCatStorer.class.getName() + "('b=2010-10-11')"); + + server.registerQuery("a = load '" + warehouseDir + "/" + tblName + "/b=2010-10-11' using PigStorage() as (a:chararray);"); + Iterator itr = server.openIterator("a"); + boolean result = compareWithFile(itr, anyExistingFileInCurDir, 1, "2010-10-11", ":hello"); + assertTrue(result); + } +} diff --git hcatalog/server-extensions/src/main/java/org/apache/hcatalog/listener/NotificationListener.java hcatalog/server-extensions/src/main/java/org/apache/hcatalog/listener/NotificationListener.java new file mode 100644 index 0000000..0c495a0 --- /dev/null +++ hcatalog/server-extensions/src/main/java/org/apache/hcatalog/listener/NotificationListener.java @@ -0,0 +1,381 @@ +/** + * 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.hcatalog.listener; + +import java.util.ArrayList; +import java.util.HashMap; + +import javax.jms.Connection; +import javax.jms.ConnectionFactory; +import javax.jms.Destination; +import javax.jms.ExceptionListener; +import javax.jms.IllegalStateException; +import javax.jms.JMSException; +import javax.jms.Message; +import javax.jms.MessageProducer; +import javax.jms.Session; +import javax.jms.Topic; +import javax.naming.Context; +import javax.naming.InitialContext; +import javax.naming.NamingException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler; +import org.apache.hadoop.hive.metastore.MetaStoreEventListener; +import org.apache.hadoop.hive.metastore.api.InvalidOperationException; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hadoop.hive.metastore.api.Order; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.metastore.events.AddPartitionEvent; +import org.apache.hadoop.hive.metastore.events.AlterPartitionEvent; +import org.apache.hadoop.hive.metastore.events.AlterTableEvent; +import org.apache.hadoop.hive.metastore.events.CreateDatabaseEvent; +import org.apache.hadoop.hive.metastore.events.CreateTableEvent; +import org.apache.hadoop.hive.metastore.events.DropDatabaseEvent; +import org.apache.hadoop.hive.metastore.events.DropPartitionEvent; +import org.apache.hadoop.hive.metastore.events.DropTableEvent; +import org.apache.hadoop.hive.metastore.events.ListenerEvent; +import org.apache.hadoop.hive.metastore.events.LoadPartitionDoneEvent; +import org.apache.hcatalog.common.HCatConstants; +import org.apache.hcatalog.messaging.HCatEventMessage; +import org.apache.hcatalog.messaging.MessageFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Implementation of + * {@link org.apache.hadoop.hive.metastore.MetaStoreEventListener} It sends + * message on two type of topics. One has name of form dbName.tblName On this + * topic, two kind of messages are sent: add/drop partition and + * finalize_partition message. Second topic has name "HCAT" and messages sent on + * it are: add/drop database and add/drop table. All messages also has a + * property named "HCAT_EVENT" set on them whose value can be used to configure + * message selector on subscriber side. + * + * @deprecated Use/modify {@link org.apache.hive.hcatalog.listener.NotificationListener} instead + */ +public class NotificationListener extends MetaStoreEventListener { + + private static final Logger LOG = LoggerFactory.getLogger(NotificationListener.class); + protected Session session; + protected Connection conn; + private static MessageFactory messageFactory = MessageFactory.getInstance(); + + /** + * Create message bus connection and session in constructor. + */ + public NotificationListener(final Configuration conf) { + + super(conf); + createConnection(); + } + + private static String getTopicName(Partition partition, + ListenerEvent partitionEvent) throws MetaException { + try { + return partitionEvent.getHandler() + .get_table(partition.getDbName(), partition.getTableName()) + .getParameters().get(HCatConstants.HCAT_MSGBUS_TOPIC_NAME); + } catch (NoSuchObjectException e) { + throw new MetaException(e.toString()); + } + } + + @Override + public void onAddPartition(AddPartitionEvent partitionEvent) + throws MetaException { + // Subscriber can get notification of newly add partition in a + // particular table by listening on a topic named "dbName.tableName" + // and message selector string as "HCAT_EVENT = HCAT_ADD_PARTITION" + if (partitionEvent.getStatus()) { + + Partition partition = partitionEvent.getPartition(); + String topicName = getTopicName(partition, partitionEvent); + if (topicName != null && !topicName.equals("")) { + send(messageFactory.buildAddPartitionMessage(partitionEvent.getTable(), partition), topicName); + } else { + LOG.info("Topic name not found in metastore. Suppressing HCatalog notification for " + + partition.getDbName() + + "." + + partition.getTableName() + + " To enable notifications for this table, please do alter table set properties (" + + HCatConstants.HCAT_MSGBUS_TOPIC_NAME + + "=.) or whatever you want topic name to be."); + } + } + + } + + /** + * Send dropped partition notifications. Subscribers can receive these notifications for a + * particular table by listening on a topic named "dbName.tableName" with message selector + * string {@value org.apache.hcatalog.common.HCatConstants#HCAT_EVENT} = + * {@value org.apache.hcatalog.common.HCatConstants#HCAT_DROP_PARTITION_EVENT}. + *
+ * TODO: DataNucleus 2.0.3, currently used by the HiveMetaStore for persistence, has been + * found to throw NPE when serializing objects that contain null. For this reason we override + * some fields in the StorageDescriptor of this notification. This should be fixed after + * HIVE-2084 "Upgrade datanucleus from 2.0.3 to 3.0.1" is resolved. + */ + @Override + public void onDropPartition(DropPartitionEvent partitionEvent) throws MetaException { + if (partitionEvent.getStatus()) { + Partition partition = partitionEvent.getPartition(); + StorageDescriptor sd = partition.getSd(); + sd.setBucketCols(new ArrayList()); + sd.setSortCols(new ArrayList()); + sd.setParameters(new HashMap()); + sd.getSerdeInfo().setParameters(new HashMap()); + sd.getSkewedInfo().setSkewedColNames(new ArrayList()); + String topicName = getTopicName(partition, partitionEvent); + if (topicName != null && !topicName.equals("")) { + send(messageFactory.buildDropPartitionMessage(partitionEvent.getTable(), partition), topicName); + } else { + LOG.info("Topic name not found in metastore. Suppressing HCatalog notification for " + + partition.getDbName() + + "." + + partition.getTableName() + + " To enable notifications for this table, please do alter table set properties (" + + HCatConstants.HCAT_MSGBUS_TOPIC_NAME + + "=.) or whatever you want topic name to be."); + } + } + } + + @Override + public void onCreateDatabase(CreateDatabaseEvent dbEvent) + throws MetaException { + // Subscriber can get notification about addition of a database in HCAT + // by listening on a topic named "HCAT" and message selector string + // as "HCAT_EVENT = HCAT_ADD_DATABASE" + if (dbEvent.getStatus()) { + String topicName = getTopicPrefix(dbEvent.getHandler().getHiveConf()); + send(messageFactory.buildCreateDatabaseMessage(dbEvent.getDatabase()), topicName); + } + } + + @Override + public void onDropDatabase(DropDatabaseEvent dbEvent) throws MetaException { + // Subscriber can get notification about drop of a database in HCAT + // by listening on a topic named "HCAT" and message selector string + // as "HCAT_EVENT = HCAT_DROP_DATABASE" + if (dbEvent.getStatus()) { + String topicName = getTopicPrefix(dbEvent.getHandler().getHiveConf()); + send(messageFactory.buildDropDatabaseMessage(dbEvent.getDatabase()), topicName); + } + } + + @Override + public void onCreateTable(CreateTableEvent tableEvent) throws MetaException { + // Subscriber can get notification about addition of a table in HCAT + // by listening on a topic named "HCAT" and message selector string + // as "HCAT_EVENT = HCAT_ADD_TABLE" + if (tableEvent.getStatus()) { + Table tbl = tableEvent.getTable(); + HMSHandler handler = tableEvent.getHandler(); + HiveConf conf = handler.getHiveConf(); + Table newTbl; + try { + newTbl = handler.get_table(tbl.getDbName(), tbl.getTableName()) + .deepCopy(); + newTbl.getParameters().put( + HCatConstants.HCAT_MSGBUS_TOPIC_NAME, + getTopicPrefix(conf) + "." + newTbl.getDbName().toLowerCase() + "." + + newTbl.getTableName().toLowerCase()); + handler.alter_table(newTbl.getDbName(), newTbl.getTableName(), newTbl); + } catch (InvalidOperationException e) { + MetaException me = new MetaException(e.toString()); + me.initCause(e); + throw me; + } catch (NoSuchObjectException e) { + MetaException me = new MetaException(e.toString()); + me.initCause(e); + throw me; + } + String topicName = getTopicPrefix(conf) + "." + newTbl.getDbName().toLowerCase(); + send(messageFactory.buildCreateTableMessage(newTbl), topicName); + } + } + + private String getTopicPrefix(HiveConf conf) { + return conf.get(HCatConstants.HCAT_MSGBUS_TOPIC_PREFIX, + HCatConstants.HCAT_DEFAULT_TOPIC_PREFIX); + } + + /** + * Send dropped table notifications. Subscribers can receive these notifications for + * dropped tables by listening on topic "HCAT" with message selector string + * {@value org.apache.hcatalog.common.HCatConstants#HCAT_EVENT} = + * {@value org.apache.hcatalog.common.HCatConstants#HCAT_DROP_TABLE_EVENT} + *
+ * TODO: DataNucleus 2.0.3, currently used by the HiveMetaStore for persistence, has been + * found to throw NPE when serializing objects that contain null. For this reason we override + * some fields in the StorageDescriptor of this notification. This should be fixed after + * HIVE-2084 "Upgrade datanucleus from 2.0.3 to 3.0.1" is resolved. + */ + @Override + public void onDropTable(DropTableEvent tableEvent) throws MetaException { + // Subscriber can get notification about drop of a table in HCAT + // by listening on a topic named "HCAT" and message selector string + // as "HCAT_EVENT = HCAT_DROP_TABLE" + + // Datanucleus throws NPE when we try to serialize a table object + // retrieved from metastore. To workaround that we reset following objects + + if (tableEvent.getStatus()) { + Table table = tableEvent.getTable(); + String topicName = getTopicPrefix(tableEvent.getHandler().getHiveConf()) + "." + table.getDbName().toLowerCase(); + send(messageFactory.buildDropTableMessage(table), topicName); + } + } + + /** + * @param hCatEventMessage The HCatEventMessage being sent over JMS. + * @param topicName is the name on message broker on which message is sent. + */ + protected void send(HCatEventMessage hCatEventMessage, String topicName) { + try { + if(null == session){ + // this will happen, if we never able to establish a connection. + createConnection(); + if (null == session){ + // Still not successful, return from here. + LOG.error("Invalid session. Failed to send message on topic: " + + topicName + " event: " + hCatEventMessage.getEventType()); + return; + } + } + + Destination topic = getTopic(topicName); + + if (null == topic){ + // Still not successful, return from here. + LOG.error("Invalid session. Failed to send message on topic: " + + topicName + " event: " + hCatEventMessage.getEventType()); + return; + } + + MessageProducer producer = session.createProducer(topic); + Message msg = session.createTextMessage(hCatEventMessage.toString()); + + msg.setStringProperty(HCatConstants.HCAT_EVENT, hCatEventMessage.getEventType().toString()); + msg.setStringProperty(HCatConstants.HCAT_MESSAGE_VERSION, messageFactory.getVersion()); + msg.setStringProperty(HCatConstants.HCAT_MESSAGE_FORMAT, messageFactory.getMessageFormat()); + producer.send(msg); + // Message must be transacted before we return. + session.commit(); + } + catch(Exception e){ + // Gobble up the exception. Message delivery is best effort. + LOG.error("Failed to send message on topic: " + topicName + + " event: " + hCatEventMessage.getEventType(), e); + } + } + + /** + * Get the topic object for the topicName, it also tries to reconnect + * if the connection appears to be broken. + * + * @param topicName The String identifying the message-topic. + * @return A {@link Topic} object corresponding to the specified topicName. + * @throws JMSException + */ + protected Topic getTopic(final String topicName) throws JMSException { + Topic topic; + try { + // Topics are created on demand. If it doesn't exist on broker it will + // be created when broker receives this message. + topic = session.createTopic(topicName); + } catch (IllegalStateException ise) { + // this will happen if we were able to establish connection once, but its no longer valid, + // ise is thrown, catch it and retry. + LOG.error("Seems like connection is lost. Retrying", ise); + createConnection(); + topic = session.createTopic(topicName); + } + return topic; + } + + protected void createConnection() { + + Context jndiCntxt; + try { + jndiCntxt = new InitialContext(); + ConnectionFactory connFac = (ConnectionFactory) jndiCntxt + .lookup("ConnectionFactory"); + Connection conn = connFac.createConnection(); + conn.start(); + conn.setExceptionListener(new ExceptionListener() { + @Override + public void onException(JMSException jmse) { + LOG.error(jmse.toString()); + } + }); + // We want message to be sent when session commits, thus we run in + // transacted mode. + session = conn.createSession(true, Session.SESSION_TRANSACTED); + } catch (NamingException e) { + LOG.error("JNDI error while setting up Message Bus connection. " + + "Please make sure file named 'jndi.properties' is in " + + "classpath and contains appropriate key-value pairs.", e); + } catch (JMSException e) { + LOG.error("Failed to initialize connection to message bus", e); + } catch (Throwable t) { + LOG.error("Unable to connect to JMS provider", t); + } + } + + @Override + protected void finalize() throws Throwable { + // Close the connection before dying. + try { + if (null != session) + session.close(); + if (conn != null) { + conn.close(); + } + + } catch (Exception ignore) { + LOG.info("Failed to close message bus connection.", ignore); + } + } + + @Override + public void onLoadPartitionDone(LoadPartitionDoneEvent lpde) + throws MetaException { +// TODO: Fix LoadPartitionDoneEvent. Currently, LPDE can only carry a single partition-spec. And that defeats the purpose. +// if(lpde.getStatus()) +// send(lpde.getPartitionName(),lpde.getTable().getParameters().get(HCatConstants.HCAT_MSGBUS_TOPIC_NAME),HCatConstants.HCAT_PARTITION_DONE_EVENT); + } + + @Override + public void onAlterPartition(AlterPartitionEvent ape) throws MetaException { + // no-op + } + + @Override + public void onAlterTable(AlterTableEvent ate) throws MetaException { + // no-op + } +} diff --git hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/AddPartitionMessage.java hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/AddPartitionMessage.java new file mode 100644 index 0000000..1a6e868 --- /dev/null +++ hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/AddPartitionMessage.java @@ -0,0 +1,55 @@ +/** + * 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.hcatalog.messaging; + +import java.util.List; +import java.util.Map; + +/** + * The HCat message sent when partition(s) are added to a table. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.messaging.AddPartitionMessage} instead + */ +public abstract class AddPartitionMessage extends HCatEventMessage { + + protected AddPartitionMessage() { + super(EventType.ADD_PARTITION); + } + + /** + * Getter for name of table (where partitions are added). + * @return Table-name (String). + */ + public abstract String getTable(); + + /** + * Getter for list of partitions added. + * @return List of maps, where each map identifies values for each partition-key, for every added partition. + */ + public abstract List> getPartitions (); + + @Override + public HCatEventMessage checkValid() { + if (getTable() == null) + throw new IllegalStateException("Table name unset."); + if (getPartitions() == null) + throw new IllegalStateException("Partition-list unset."); + return super.checkValid(); + } +} diff --git hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/CreateDatabaseMessage.java hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/CreateDatabaseMessage.java new file mode 100644 index 0000000..76b0702 --- /dev/null +++ hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/CreateDatabaseMessage.java @@ -0,0 +1,32 @@ +/** + * 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.hcatalog.messaging; + +/** + * HCat message sent when a Database is created in HCatalog. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.messaging.CreateDatabaseMessage} instead + */ +public abstract class CreateDatabaseMessage extends HCatEventMessage { + + protected CreateDatabaseMessage() { + super(EventType.CREATE_DATABASE); + } + +} diff --git hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/CreateTableMessage.java hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/CreateTableMessage.java new file mode 100644 index 0000000..e97a35e --- /dev/null +++ hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/CreateTableMessage.java @@ -0,0 +1,44 @@ +/** + * 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.hcatalog.messaging; + +/** + * HCat message sent when a table is created in HCatalog. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.messaging.CreateTableMessage} instead + */ +public abstract class CreateTableMessage extends HCatEventMessage { + + protected CreateTableMessage() { + super(EventType.CREATE_TABLE); + } + + /** + * Getter for the name of table created in HCatalog. + * @return Table-name (String). + */ + public abstract String getTable(); + + @Override + public HCatEventMessage checkValid() { + if (getTable() == null) + throw new IllegalStateException("Table name unset."); + return super.checkValid(); + } +} diff --git hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/DropDatabaseMessage.java hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/DropDatabaseMessage.java new file mode 100644 index 0000000..7d231c5 --- /dev/null +++ hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/DropDatabaseMessage.java @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hcatalog.messaging; + +/** + * HCat message sent when a Database is dropped from HCatalog. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.messaging.DropDatabaseMessage} instead + */ +public abstract class DropDatabaseMessage extends HCatEventMessage { + + protected DropDatabaseMessage() { + super(EventType.DROP_DATABASE); + } +} diff --git hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/DropPartitionMessage.java hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/DropPartitionMessage.java new file mode 100644 index 0000000..829e629 --- /dev/null +++ hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/DropPartitionMessage.java @@ -0,0 +1,46 @@ +/** + * 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.hcatalog.messaging; + +import java.util.List; +import java.util.Map; + +/** + * HCat message sent when a partition is dropped in HCatalog. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.messaging.DropPartitionMessage} instead + */ +public abstract class DropPartitionMessage extends HCatEventMessage { + + protected DropPartitionMessage() { + super(EventType.DROP_PARTITION); + } + + public abstract String getTable(); + public abstract List> getPartitions (); + + @Override + public HCatEventMessage checkValid() { + if (getTable() == null) + throw new IllegalStateException("Table name unset."); + if (getPartitions() == null) + throw new IllegalStateException("Partition-list unset."); + return super.checkValid(); + } +} diff --git hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/DropTableMessage.java hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/DropTableMessage.java new file mode 100644 index 0000000..8b2fcbc --- /dev/null +++ hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/DropTableMessage.java @@ -0,0 +1,44 @@ +/** + * 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.hcatalog.messaging; + +/** + * HCat message sent when a Table is dropped in HCatalog. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.messaging.DropTableMessage} instead + */ +public abstract class DropTableMessage extends HCatEventMessage { + + protected DropTableMessage() { + super(EventType.DROP_TABLE); + } + + /** + * Getter for the name of the table being dropped. + * @return Table-name (String). + */ + public abstract String getTable(); + + @Override + public HCatEventMessage checkValid() { + if (getTable() == null) + throw new IllegalStateException("Table name unset."); + return super.checkValid(); + } +} diff --git hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/HCatEventMessage.java hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/HCatEventMessage.java new file mode 100644 index 0000000..af67f16 --- /dev/null +++ hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/HCatEventMessage.java @@ -0,0 +1,101 @@ +/** + * 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.hcatalog.messaging; + +import org.apache.hcatalog.common.HCatConstants; + +/** + * Class representing messages emitted when Metastore operations are done. + * (E.g. Creation and deletion of databases, tables and partitions.) + * @deprecated Use/modify {@link org.apache.hive.hcatalog.messaging.HCatEventMessage} instead + */ +public abstract class HCatEventMessage { + + /** + * Enumeration of all supported types of Metastore operations. + */ + public static enum EventType { + + CREATE_DATABASE(HCatConstants.HCAT_CREATE_DATABASE_EVENT), + DROP_DATABASE(HCatConstants.HCAT_DROP_DATABASE_EVENT), + CREATE_TABLE(HCatConstants.HCAT_CREATE_TABLE_EVENT), + DROP_TABLE(HCatConstants.HCAT_DROP_TABLE_EVENT), + ADD_PARTITION(HCatConstants.HCAT_ADD_PARTITION_EVENT), + DROP_PARTITION(HCatConstants.HCAT_DROP_PARTITION_EVENT); + + private String typeString; + + EventType(String typeString) { + this.typeString = typeString; + } + + @Override + public String toString() { return typeString; } + } + + protected EventType eventType; + + protected HCatEventMessage(EventType eventType) { + this.eventType = eventType; + } + + public EventType getEventType() { + return eventType; + } + + /** + * Getter for HCatalog Server's URL. + * (This is where the event originates from.) + * @return HCatalog Server's URL (String). + */ + public abstract String getServer(); + + /** + * Getter for the Kerberos principal of the HCatalog service. + * @return HCatalog Service Principal (String). + */ + public abstract String getServicePrincipal(); + + /** + * Getter for the name of the Database on which the Metastore operation is done. + * @return Database-name (String). + */ + public abstract String getDB(); + + /** + * Getter for the timestamp associated with the operation. + * @return Timestamp (Long - seconds since epoch). + */ + public abstract Long getTimestamp(); + + /** + * Class invariant. Checked after construction or deserialization. + */ + public HCatEventMessage checkValid() { + if (getServer() == null || getServicePrincipal() == null) + throw new IllegalStateException("Server-URL/Service-Principal shouldn't be null."); + if (getEventType() == null) + throw new IllegalStateException("Event-type unset."); + if (getDB() == null) + throw new IllegalArgumentException("DB-name unset."); + + return this; + } +} diff --git hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/MessageDeserializer.java hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/MessageDeserializer.java new file mode 100644 index 0000000..1012172 --- /dev/null +++ hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/MessageDeserializer.java @@ -0,0 +1,84 @@ +/** + * 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.hcatalog.messaging; + +/** + * Interface for converting HCat events from String-form back to HCatEventMessage instances. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.messaging.MessageDeserializer} instead + */ +public abstract class MessageDeserializer { + + /** + * Method to construct HCatEventMessage from string. + */ + public HCatEventMessage getHCatEventMessage(String eventTypeString, String messageBody) { + + switch (HCatEventMessage.EventType.valueOf(eventTypeString)) { + case CREATE_DATABASE: + return getCreateDatabaseMessage(messageBody); + case DROP_DATABASE: + return getDropDatabaseMessage(messageBody); + case CREATE_TABLE: + return getCreateTableMessage(messageBody); + case DROP_TABLE: + return getDropTableMessage(messageBody); + case ADD_PARTITION: + return getAddPartitionMessage(messageBody); + case DROP_PARTITION: + return getDropPartitionMessage(messageBody); + + default: + throw new IllegalArgumentException("Unsupported event-type: " + eventTypeString); + } + } + + /** + * Method to de-serialize CreateDatabaseMessage instance. + */ + public abstract CreateDatabaseMessage getCreateDatabaseMessage(String messageBody); + + /** + * Method to de-serialize DropDatabaseMessage instance. + */ + public abstract DropDatabaseMessage getDropDatabaseMessage(String messageBody); + + /** + * Method to de-serialize CreateTableMessage instance. + */ + public abstract CreateTableMessage getCreateTableMessage(String messageBody); + + /** + * Method to de-serialize DropTableMessage instance. + */ + public abstract DropTableMessage getDropTableMessage(String messageBody); + + /** + * Method to de-serialize AddPartitionMessage instance. + */ + public abstract AddPartitionMessage getAddPartitionMessage(String messageBody); + + /** + * Method to de-serialize DropPartitionMessage instance. + */ + public abstract DropPartitionMessage getDropPartitionMessage(String messageBody); + + // Protection against construction. + protected MessageDeserializer() {} +} diff --git hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/MessageFactory.java hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/MessageFactory.java new file mode 100644 index 0000000..36da95d --- /dev/null +++ hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/MessageFactory.java @@ -0,0 +1,139 @@ +/** + * 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.hcatalog.messaging; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.util.ReflectionUtils; +import org.apache.hcatalog.messaging.json.JSONMessageFactory; + +/** + * Abstract Factory for the construction of HCatalog message instances. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.messaging.MessageFactory} instead + */ +public abstract class MessageFactory { + + private static MessageFactory instance = new JSONMessageFactory(); + + protected static final HiveConf hiveConf = new HiveConf(); + static { + hiveConf.addResource("hive-site.xml"); + } + + private static final String CONF_LABEL_HCAT_MESSAGE_FACTORY_IMPL_PREFIX = "hcatalog.message.factory.impl."; + private static final String CONF_LABEL_HCAT_MESSAGE_FORMAT = "hcatalog.message.format"; + private static final String HCAT_MESSAGE_FORMAT = hiveConf.get(CONF_LABEL_HCAT_MESSAGE_FORMAT, "json"); + private static final String DEFAULT_MESSAGE_FACTORY_IMPL = "org.apache.hcatalog.messaging.json.JSONMessageFactory"; + private static final String HCAT_MESSAGE_FACTORY_IMPL = hiveConf.get(CONF_LABEL_HCAT_MESSAGE_FACTORY_IMPL_PREFIX + + HCAT_MESSAGE_FORMAT, + DEFAULT_MESSAGE_FACTORY_IMPL); + + protected static final String HCAT_SERVER_URL = hiveConf.get(HiveConf.ConfVars.METASTOREURIS.name(), ""); + protected static final String HCAT_SERVICE_PRINCIPAL = hiveConf.get(HiveConf.ConfVars.METASTORE_KERBEROS_PRINCIPAL.name(), ""); + + /** + * Getter for MessageFactory instance. + */ + public static MessageFactory getInstance() { + if (instance == null) { + instance = getInstance(HCAT_MESSAGE_FACTORY_IMPL); + } + return instance; + } + + private static MessageFactory getInstance(String className) { + try { + return (MessageFactory)ReflectionUtils.newInstance(Class.forName(className), hiveConf); + } + catch (ClassNotFoundException classNotFound) { + throw new IllegalStateException("Could not construct MessageFactory implementation: ", classNotFound); + } + } + + /** + * Getter for MessageDeserializer, corresponding to the specified format and version. + * @param format Serialization format for notifications. + * @param version Version of serialization format (currently ignored.) + * @return MessageDeserializer. + */ + public static MessageDeserializer getDeserializer(String format, + String version) { + return getInstance(hiveConf.get(CONF_LABEL_HCAT_MESSAGE_FACTORY_IMPL_PREFIX + format, + DEFAULT_MESSAGE_FACTORY_IMPL)).getDeserializer(); + } + + public abstract MessageDeserializer getDeserializer(); + + /** + * Getter for version-string, corresponding to all constructed messages. + */ + public abstract String getVersion(); + + /** + * Getter for message-format. + */ + public abstract String getMessageFormat(); + + /** + * Factory method for CreateDatabaseMessage. + * @param db The Database being added. + * @return CreateDatabaseMessage instance. + */ + public abstract CreateDatabaseMessage buildCreateDatabaseMessage(Database db); + + /** + * Factory method for DropDatabaseMessage. + * @param db The Database being dropped. + * @return DropDatabaseMessage instance. + */ + public abstract DropDatabaseMessage buildDropDatabaseMessage(Database db); + + /** + * Factory method for CreateTableMessage. + * @param table The Table being created. + * @return CreateTableMessage instance. + */ + public abstract CreateTableMessage buildCreateTableMessage(Table table); + + /** + * Factory method for DropTableMessage. + * @param table The Table being dropped. + * @return DropTableMessage instance. + */ + public abstract DropTableMessage buildDropTableMessage(Table table); + + /** + * Factory method for AddPartitionMessage. + * @param table The Table to which the partition is added. + * @param partition The Partition being added. + * @return AddPartitionMessage instance. + */ + public abstract AddPartitionMessage buildAddPartitionMessage(Table table, Partition partition); + + /** + * Factory method for DropPartitionMessage. + * @param table The Table from which the partition is dropped. + * @param partition The Partition being dropped. + * @return DropPartitionMessage instance. + */ + public abstract DropPartitionMessage buildDropPartitionMessage(Table table, Partition partition); +} diff --git hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/jms/MessagingUtils.java hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/jms/MessagingUtils.java new file mode 100644 index 0000000..b8d3050 --- /dev/null +++ hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/jms/MessagingUtils.java @@ -0,0 +1,63 @@ +/** + * 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.hcatalog.messaging.jms; + +import org.apache.commons.lang.StringUtils; +import org.apache.hcatalog.common.HCatConstants; +import org.apache.hcatalog.messaging.HCatEventMessage; +import org.apache.hcatalog.messaging.MessageFactory; + +import javax.jms.JMSException; +import javax.jms.Message; +import javax.jms.TextMessage; + +/** + * Helper Utility to assist consumers of HCat Messages in extracting + * message-content from JMS messages. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.messaging.jms.MessagingUtils} instead + */ +public class MessagingUtils { + + /** + * Method to return HCatEventMessage contained in the JMS message. + * @param message The JMS Message instance + * @return The contained HCatEventMessage + */ + public static HCatEventMessage getMessage(Message message) { + try { + String messageBody = ((TextMessage)message).getText(); + String eventType = message.getStringProperty(HCatConstants.HCAT_EVENT); + String messageVersion = message.getStringProperty(HCatConstants.HCAT_MESSAGE_VERSION); + String messageFormat = message.getStringProperty(HCatConstants.HCAT_MESSAGE_FORMAT); + + if (StringUtils.isEmpty(messageBody) || StringUtils.isEmpty(eventType)) + throw new IllegalArgumentException("Could not extract HCatEventMessage. " + + "EventType and/or MessageBody is null/empty."); + + return MessageFactory.getDeserializer(messageFormat, messageVersion).getHCatEventMessage(eventType, messageBody); + } + catch (JMSException exception) { + throw new IllegalArgumentException("Could not extract HCatEventMessage. ", exception); + } + } + + // Prevent construction. + private MessagingUtils() {} +} diff --git hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/json/JSONAddPartitionMessage.java hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/json/JSONAddPartitionMessage.java new file mode 100644 index 0000000..a7d0348 --- /dev/null +++ hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/json/JSONAddPartitionMessage.java @@ -0,0 +1,86 @@ +/** + * 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.hcatalog.messaging.json; + +import org.apache.hcatalog.messaging.AddPartitionMessage; +import org.codehaus.jackson.annotate.JsonProperty; + +import java.util.List; +import java.util.Map; + +/** + * JSON implementation of AddPartitionMessage. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.messaging.jms.MessagingUtils} instead + */ +public class JSONAddPartitionMessage extends AddPartitionMessage { + + @JsonProperty + String server, servicePrincipal, db, table; + + @JsonProperty + Long timestamp; + + @JsonProperty + List> partitions; + + /** + * Default Constructor. Required for Jackson. + */ + public JSONAddPartitionMessage() {} + + public JSONAddPartitionMessage(String server, String servicePrincipal, String db, String table, + List> partitions, Long timestamp) { + this.server = server; + this.servicePrincipal = servicePrincipal; + this.db = db; + this.table = table; + this.partitions = partitions; + this.timestamp = timestamp; + checkValid(); + } + + @Override + public String getServer() { return server; } + + @Override + public String getServicePrincipal() { return servicePrincipal; } + + @Override + public String getDB() { return db; } + + @Override + public String getTable() { return table; } + + @Override + public Long getTimestamp() { return timestamp; } + + @Override + public List> getPartitions () { return partitions; } + + @Override + public String toString() { + try { + return JSONMessageDeserializer.mapper.writeValueAsString(this); + } + catch (Exception exception) { + throw new IllegalArgumentException("Could not serialize: ", exception); + } + } +} diff --git hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/json/JSONCreateDatabaseMessage.java hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/json/JSONCreateDatabaseMessage.java new file mode 100644 index 0000000..1e28082 --- /dev/null +++ hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/json/JSONCreateDatabaseMessage.java @@ -0,0 +1,72 @@ +/** + * 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.hcatalog.messaging.json; + +import org.apache.hcatalog.messaging.CreateDatabaseMessage; +import org.codehaus.jackson.annotate.JsonProperty; + +/** + * JSON Implementation of CreateDatabaseMessage. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.messaging.json.JSONCreateDatabaseMessage} instead + */ +public class JSONCreateDatabaseMessage extends CreateDatabaseMessage { + + @JsonProperty + String server, servicePrincipal, db; + + @JsonProperty + Long timestamp; + + /** + * Default constructor, required for Jackson. + */ + public JSONCreateDatabaseMessage() {} + + public JSONCreateDatabaseMessage(String server, String servicePrincipal, String db, Long timestamp) { + this.server = server; + this.servicePrincipal = servicePrincipal; + this.db = db; + this.timestamp = timestamp; + checkValid(); + } + + @Override + public String getDB() { return db; } + + @Override + public String getServer() { return server; } + + @Override + public String getServicePrincipal() { return servicePrincipal; } + + @Override + public Long getTimestamp() { return timestamp; } + + @Override + public String toString() { + try { + return JSONMessageDeserializer.mapper.writeValueAsString(this); + } + catch (Exception exception) { + throw new IllegalArgumentException("Could not serialize: ", exception); + } + } + +} diff --git hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/json/JSONCreateTableMessage.java hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/json/JSONCreateTableMessage.java new file mode 100644 index 0000000..35e1e4e --- /dev/null +++ hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/json/JSONCreateTableMessage.java @@ -0,0 +1,75 @@ +/** + * 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.hcatalog.messaging.json; + +import org.apache.hcatalog.messaging.CreateTableMessage; +import org.codehaus.jackson.annotate.JsonProperty; + +/** + * JSON implementation of CreateTableMessage. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.messaging.json.JSONCreateTableMessage} instead + */ +public class JSONCreateTableMessage extends CreateTableMessage { + + @JsonProperty + String server, servicePrincipal, db, table; + + @JsonProperty + Long timestamp; + + /** + * Default constructor, needed for Jackson. + */ + public JSONCreateTableMessage() {} + + public JSONCreateTableMessage(String server, String servicePrincipal, String db, String table, Long timestamp) { + this.server = server; + this.servicePrincipal = servicePrincipal; + this.db = db; + this.table = table; + this.timestamp = timestamp; + checkValid(); + } + + @Override + public String getServer() { return server; } + + @Override + public String getServicePrincipal() { return servicePrincipal; } + + @Override + public String getDB() { return db; } + + @Override + public Long getTimestamp() { return timestamp; } + + @Override + public String getTable() { return table; } + + @Override + public String toString() { + try { + return JSONMessageDeserializer.mapper.writeValueAsString(this); + } + catch (Exception exception) { + throw new IllegalArgumentException("Could not serialize: ", exception); + } + } +} diff --git hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/json/JSONDropDatabaseMessage.java hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/json/JSONDropDatabaseMessage.java new file mode 100644 index 0000000..c567949 --- /dev/null +++ hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/json/JSONDropDatabaseMessage.java @@ -0,0 +1,72 @@ +/** + * 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.hcatalog.messaging.json; + +import org.apache.hcatalog.messaging.DropDatabaseMessage; +import org.codehaus.jackson.annotate.JsonProperty; + +/** + * JSON implementation of DropDatabaseMessage. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.messaging.json.JSONDropDatabaseMessage} instead + */ +public class JSONDropDatabaseMessage extends DropDatabaseMessage { + + @JsonProperty + String server, servicePrincipal, db; + + @JsonProperty + Long timestamp; + + /** + * Default constructor, required for Jackson. + */ + public JSONDropDatabaseMessage() {} + + public JSONDropDatabaseMessage(String server, String servicePrincipal, String db, Long timestamp) { + this.server = server; + this.servicePrincipal = servicePrincipal; + this.db = db; + this.timestamp = timestamp; + checkValid(); + } + + + @Override + public String getServer() { return server; } + + @Override + public String getServicePrincipal() { return servicePrincipal; } + + @Override + public String getDB() { return db; } + + @Override + public Long getTimestamp() { return timestamp; } + + @Override + public String toString() { + try { + return JSONMessageDeserializer.mapper.writeValueAsString(this); + } + catch (Exception exception) { + throw new IllegalArgumentException("Could not serialize: ", exception); + } + } +} diff --git hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/json/JSONDropPartitionMessage.java hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/json/JSONDropPartitionMessage.java new file mode 100644 index 0000000..0e418aa --- /dev/null +++ hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/json/JSONDropPartitionMessage.java @@ -0,0 +1,87 @@ +/** + * 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.hcatalog.messaging.json; + +import org.apache.hcatalog.messaging.DropPartitionMessage; +import org.codehaus.jackson.annotate.JsonProperty; + +import java.util.List; +import java.util.Map; + +/** + * JSON implementation of DropPartitionMessage. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.messaging.json.JSONDropPartitionMessage} instead + */ +public class JSONDropPartitionMessage extends DropPartitionMessage { + + @JsonProperty + String server, servicePrincipal, db, table; + + @JsonProperty + Long timestamp; + + @JsonProperty + List> partitions; + + /** + * Default Constructor. Required for Jackson. + */ + public JSONDropPartitionMessage() {} + + public JSONDropPartitionMessage(String server, String servicePrincipal, String db, String table, + List> partitions, Long timestamp) { + this.server = server; + this.servicePrincipal = servicePrincipal; + this.db = db; + this.table = table; + this.partitions = partitions; + this.timestamp = timestamp; + checkValid(); + } + + + @Override + public String getServer() { return server; } + + @Override + public String getServicePrincipal() { return servicePrincipal; } + + @Override + public String getDB() { return db; } + + @Override + public String getTable() { return table; } + + @Override + public Long getTimestamp() { return timestamp; } + + @Override + public List> getPartitions () { return partitions; } + + @Override + public String toString() { + try { + return JSONMessageDeserializer.mapper.writeValueAsString(this); + } + catch (Exception exception) { + throw new IllegalArgumentException("Could not serialize: ", exception); + } + } +} diff --git hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/json/JSONDropTableMessage.java hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/json/JSONDropTableMessage.java new file mode 100644 index 0000000..505dd94 --- /dev/null +++ hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/json/JSONDropTableMessage.java @@ -0,0 +1,77 @@ +/** + * 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.hcatalog.messaging.json; + +import org.apache.hcatalog.messaging.DropTableMessage; +import org.codehaus.jackson.annotate.JsonProperty; + +/** + * JSON implementation of DropTableMessage. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.messaging.json.JSONDropTableMessage} instead + */ +public class JSONDropTableMessage extends DropTableMessage { + + @JsonProperty + String server, servicePrincipal, db, table; + + @JsonProperty + Long timestamp; + + /** + * Default constructor, needed for Jackson. + */ + public JSONDropTableMessage() {} + + public JSONDropTableMessage(String server, String servicePrincipal, String db, String table, Long timestamp) { + this.server = server; + this.servicePrincipal = servicePrincipal; + this.db = db; + this.table = table; + this.timestamp = timestamp; + checkValid(); + } + + + @Override + public String getTable() { return table; } + + @Override + public String getServer() { return server; } + + @Override + public String getServicePrincipal() { return servicePrincipal; } + + @Override + public String getDB() { return db; } + + @Override + public Long getTimestamp() { return timestamp; } + + @Override + public String toString() { + try { + return JSONMessageDeserializer.mapper.writeValueAsString(this); + } + catch (Exception exception) { + throw new IllegalArgumentException("Could not serialize: ", exception); + } + } + +} diff --git hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/json/JSONMessageDeserializer.java hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/json/JSONMessageDeserializer.java new file mode 100644 index 0000000..6fdf7d0 --- /dev/null +++ hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/json/JSONMessageDeserializer.java @@ -0,0 +1,103 @@ +/** + * 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.hcatalog.messaging.json; + +import org.apache.hcatalog.messaging.AddPartitionMessage; +import org.apache.hcatalog.messaging.CreateDatabaseMessage; +import org.apache.hcatalog.messaging.CreateTableMessage; +import org.apache.hcatalog.messaging.DropDatabaseMessage; +import org.apache.hcatalog.messaging.DropPartitionMessage; +import org.apache.hcatalog.messaging.DropTableMessage; +import org.apache.hcatalog.messaging.MessageDeserializer; +import org.codehaus.jackson.map.DeserializationConfig; +import org.codehaus.jackson.map.ObjectMapper; + +/** + * MessageDeserializer implementation, for deserializing from JSON strings. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.messaging.json.JSONMessageDeserializer} instead + */ +public class JSONMessageDeserializer extends MessageDeserializer { + + static ObjectMapper mapper = new ObjectMapper(); // Thread-safe. + + static { + mapper.configure(DeserializationConfig.Feature.FAIL_ON_UNKNOWN_PROPERTIES, false); + } + + @Override + public CreateDatabaseMessage getCreateDatabaseMessage(String messageBody) { + try { + return mapper.readValue(messageBody, JSONCreateDatabaseMessage.class); + } + catch (Exception exception) { + throw new IllegalArgumentException("Could not construct JSONCreateDatabaseMessage.", exception); + } + } + + @Override + public DropDatabaseMessage getDropDatabaseMessage(String messageBody) { + try { + return mapper.readValue(messageBody, JSONDropDatabaseMessage.class); + } + catch (Exception exception) { + throw new IllegalArgumentException("Could not construct JSONDropDatabaseMessage.", exception); + } + } + + @Override + public CreateTableMessage getCreateTableMessage(String messageBody) { + try { + return mapper.readValue(messageBody, JSONCreateTableMessage.class); + } + catch (Exception exception) { + throw new IllegalArgumentException("Could not construct JSONCreateTableMessage.", exception); + } + } + + @Override + public DropTableMessage getDropTableMessage(String messageBody) { + try { + return mapper.readValue(messageBody, JSONDropTableMessage.class); + } + catch (Exception exception) { + throw new IllegalArgumentException("Could not construct JSONDropTableMessage.", exception); + } + } + + @Override + public AddPartitionMessage getAddPartitionMessage(String messageBody) { + try { + return mapper.readValue(messageBody, JSONAddPartitionMessage.class); + } + catch (Exception exception) { + throw new IllegalArgumentException("Could not construct AddPartitionMessage.", exception); + } + } + + @Override + public DropPartitionMessage getDropPartitionMessage(String messageBody) { + try { + return mapper.readValue(messageBody, JSONDropPartitionMessage.class); + } + catch (Exception exception) { + throw new IllegalArgumentException("Could not construct DropPartitionMessage.", exception); + } + } +} diff --git hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/json/JSONMessageFactory.java hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/json/JSONMessageFactory.java new file mode 100644 index 0000000..85e3fdb --- /dev/null +++ hcatalog/server-extensions/src/main/java/org/apache/hcatalog/messaging/json/JSONMessageFactory.java @@ -0,0 +1,107 @@ +/** + * 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.hcatalog.messaging.json; + +import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hcatalog.messaging.AddPartitionMessage; +import org.apache.hcatalog.messaging.CreateDatabaseMessage; +import org.apache.hcatalog.messaging.CreateTableMessage; +import org.apache.hcatalog.messaging.DropDatabaseMessage; +import org.apache.hcatalog.messaging.DropPartitionMessage; +import org.apache.hcatalog.messaging.DropTableMessage; +import org.apache.hcatalog.messaging.MessageDeserializer; +import org.apache.hcatalog.messaging.MessageFactory; + +import java.util.Arrays; +import java.util.LinkedHashMap; +import java.util.Map; + +/** + * The JSON implementation of the MessageFactory. Constructs JSON implementations of + * each message-type. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.messaging.json.JSONMessageFactory} instead + */ +public class JSONMessageFactory extends MessageFactory { + + private static JSONMessageDeserializer deserializer = new JSONMessageDeserializer(); + + @Override + public MessageDeserializer getDeserializer() { + return deserializer; + } + + @Override + public String getVersion() { + return "0.1"; + } + + @Override + public String getMessageFormat() { + return "json"; + } + + @Override + public CreateDatabaseMessage buildCreateDatabaseMessage(Database db) { + return new JSONCreateDatabaseMessage(HCAT_SERVER_URL, HCAT_SERVICE_PRINCIPAL, db.getName(), + System.currentTimeMillis() / 1000); + } + + @Override + public DropDatabaseMessage buildDropDatabaseMessage(Database db) { + return new JSONDropDatabaseMessage(HCAT_SERVER_URL, HCAT_SERVICE_PRINCIPAL, db.getName(), + System.currentTimeMillis() / 1000); + } + + @Override + public CreateTableMessage buildCreateTableMessage(Table table) { + return new JSONCreateTableMessage(HCAT_SERVER_URL, HCAT_SERVICE_PRINCIPAL, table.getDbName(), + table.getTableName(), System.currentTimeMillis()/1000); + } + + @Override + public DropTableMessage buildDropTableMessage(Table table) { + return new JSONDropTableMessage(HCAT_SERVER_URL, HCAT_SERVICE_PRINCIPAL, table.getDbName(), table.getTableName(), + System.currentTimeMillis()/1000); + } + + @Override + public AddPartitionMessage buildAddPartitionMessage(Table table, Partition partition) { + return new JSONAddPartitionMessage(HCAT_SERVER_URL, HCAT_SERVICE_PRINCIPAL, partition.getDbName(), + partition.getTableName(), Arrays.asList(getPartitionKeyValues(table, partition)), + System.currentTimeMillis()/1000); + } + + @Override + public DropPartitionMessage buildDropPartitionMessage(Table table, Partition partition) { + return new JSONDropPartitionMessage(HCAT_SERVER_URL, HCAT_SERVICE_PRINCIPAL, partition.getDbName(), + partition.getTableName(), Arrays.asList(getPartitionKeyValues(table, partition)), + System.currentTimeMillis()/1000); + } + + private static Map getPartitionKeyValues(Table table, Partition partition) { + Map partitionKeys = new LinkedHashMap(); + for (int i=0; i actualMessages = new ArrayList(); + + @Before + public void setUp() throws Exception { + System.setProperty("java.naming.factory.initial", + "org.apache.activemq.jndi.ActiveMQInitialContextFactory"); + System.setProperty("java.naming.provider.url", + "vm://localhost?broker.persistent=false"); + ConnectionFactory connFac = new ActiveMQConnectionFactory( + "vm://localhost?broker.persistent=false"); + Connection conn = connFac.createConnection(); + conn.start(); + // We want message to be sent when session commits, thus we run in + // transacted mode. + Session session = conn.createSession(true, Session.SESSION_TRANSACTED); + Destination hcatTopic = session + .createTopic(HCatConstants.HCAT_DEFAULT_TOPIC_PREFIX); + MessageConsumer consumer1 = session.createConsumer(hcatTopic); + consumer1.setMessageListener(this); + Destination tblTopic = session + .createTopic(HCatConstants.HCAT_DEFAULT_TOPIC_PREFIX + ".mydb.mytbl"); + MessageConsumer consumer2 = session.createConsumer(tblTopic); + consumer2.setMessageListener(this); + Destination dbTopic = session + .createTopic(HCatConstants.HCAT_DEFAULT_TOPIC_PREFIX + ".mydb"); + MessageConsumer consumer3 = session.createConsumer(dbTopic); + consumer3.setMessageListener(this); + + setUpHiveConf(); + hiveConf.set(ConfVars.METASTORE_EVENT_LISTENERS.varname, + NotificationListener.class.getName()); + SessionState.start(new CliSessionState(hiveConf)); + driver = new Driver(hiveConf); + client = new HiveMetaStoreClient(hiveConf); + } + + @After + public void tearDown() throws Exception { + List expectedMessages = Arrays.asList( + HCatConstants.HCAT_CREATE_DATABASE_EVENT, + HCatConstants.HCAT_CREATE_TABLE_EVENT, + HCatConstants.HCAT_ADD_PARTITION_EVENT, + HCatConstants.HCAT_DROP_PARTITION_EVENT, + HCatConstants.HCAT_DROP_TABLE_EVENT, + HCatConstants.HCAT_DROP_DATABASE_EVENT); + Assert.assertEquals(expectedMessages, actualMessages); + } + + @Test + public void testAMQListener() throws Exception { + driver.run("create database mydb"); + driver.run("use mydb"); + driver.run("create table mytbl (a string) partitioned by (b string)"); + driver.run("alter table mytbl add partition(b='2011')"); + Map kvs = new HashMap(1); + kvs.put("b", "2011"); + client.markPartitionForEvent("mydb", "mytbl", kvs, + PartitionEventType.LOAD_DONE); + driver.run("alter table mytbl drop partition(b='2011')"); + driver.run("drop table mytbl"); + driver.run("drop database mydb"); + } + + @Override + public void onMessage(Message msg) { + String event; + try { + event = msg.getStringProperty(HCatConstants.HCAT_EVENT); + String format = msg.getStringProperty(HCatConstants.HCAT_MESSAGE_FORMAT); + String version = msg.getStringProperty(HCatConstants.HCAT_MESSAGE_VERSION); + String messageBody = ((TextMessage)msg).getText(); + actualMessages.add(event); + MessageDeserializer deserializer = MessageFactory.getDeserializer(format, version); + + if (event.equals(HCatConstants.HCAT_CREATE_DATABASE_EVENT)) { + + Assert.assertEquals("topic://" + HCatConstants.HCAT_DEFAULT_TOPIC_PREFIX, msg + .getJMSDestination().toString()); + CreateDatabaseMessage message = deserializer.getCreateDatabaseMessage(messageBody); + Assert.assertEquals("mydb", message.getDB()); + HCatEventMessage message2 = MessagingUtils.getMessage(msg); + Assert.assertTrue("Unexpected message-type.", message2 instanceof CreateDatabaseMessage); + Assert.assertEquals("mydb", message2.getDB()); + } else if (event.equals(HCatConstants.HCAT_CREATE_TABLE_EVENT)) { + + Assert.assertEquals("topic://hcat.mydb", msg.getJMSDestination().toString()); + CreateTableMessage message = deserializer.getCreateTableMessage(messageBody); + Assert.assertEquals("mytbl", message.getTable()); + Assert.assertEquals("mydb", message.getDB()); + HCatEventMessage message2 = MessagingUtils.getMessage(msg); + Assert.assertTrue("Unexpected message-type.", message2 instanceof CreateTableMessage); + Assert.assertEquals("mydb", message2.getDB()); + Assert.assertEquals("mytbl", ((CreateTableMessage) message2).getTable()); + } else if (event.equals(HCatConstants.HCAT_ADD_PARTITION_EVENT)) { + + Assert.assertEquals("topic://hcat.mydb.mytbl", msg.getJMSDestination() + .toString()); + AddPartitionMessage message = deserializer.getAddPartitionMessage(messageBody); + Assert.assertEquals("mytbl", message.getTable()); + Assert.assertEquals("mydb", message.getDB()); + Assert.assertEquals(1, message.getPartitions().size()); + Assert.assertEquals("2011", message.getPartitions().get(0).get("b")); + HCatEventMessage message2 = MessagingUtils.getMessage(msg); + Assert.assertTrue("Unexpected message-type.", message2 instanceof AddPartitionMessage); + Assert.assertEquals("mydb", message2.getDB()); + Assert.assertEquals("mytbl", ((AddPartitionMessage) message2).getTable()); + Assert.assertEquals(1, ((AddPartitionMessage) message2).getPartitions().size()); + Assert.assertEquals("2011", ((AddPartitionMessage) message2).getPartitions().get(0).get("b")); + } else if (event.equals(HCatConstants.HCAT_DROP_PARTITION_EVENT)) { + + Assert.assertEquals("topic://hcat.mydb.mytbl", msg.getJMSDestination() + .toString()); + DropPartitionMessage message = deserializer.getDropPartitionMessage(messageBody); + Assert.assertEquals("mytbl", message.getTable()); + Assert.assertEquals("mydb", message.getDB()); + Assert.assertEquals(1, message.getPartitions().size()); + Assert.assertEquals("2011", message.getPartitions().get(0).get("b")); + HCatEventMessage message2 = MessagingUtils.getMessage(msg); + Assert.assertTrue("Unexpected message-type.", message2 instanceof DropPartitionMessage); + Assert.assertEquals("mydb", message2.getDB()); + Assert.assertEquals("mytbl", ((DropPartitionMessage) message2).getTable()); + Assert.assertEquals(1, ((DropPartitionMessage) message2).getPartitions().size()); + Assert.assertEquals("2011", ((DropPartitionMessage) message2).getPartitions().get(0).get("b")); + } else if (event.equals(HCatConstants.HCAT_DROP_TABLE_EVENT)) { + + Assert.assertEquals("topic://hcat.mydb", msg.getJMSDestination().toString()); + DropTableMessage message = deserializer.getDropTableMessage(messageBody); + Assert.assertEquals("mytbl", message.getTable()); + Assert.assertEquals("mydb", message.getDB()); + HCatEventMessage message2 = MessagingUtils.getMessage(msg); + Assert.assertTrue("Unexpected message-type.", message2 instanceof DropTableMessage); + Assert.assertEquals("mydb", message2.getDB()); + Assert.assertEquals("mytbl", ((DropTableMessage) message2).getTable()); + } else if (event.equals(HCatConstants.HCAT_DROP_DATABASE_EVENT)) { + + Assert.assertEquals("topic://" + HCatConstants.HCAT_DEFAULT_TOPIC_PREFIX, msg + .getJMSDestination().toString()); + DropDatabaseMessage message = deserializer.getDropDatabaseMessage(messageBody); + Assert.assertEquals("mydb", message.getDB()); + HCatEventMessage message2 = MessagingUtils.getMessage(msg); + Assert.assertTrue("Unexpected message-type.", message2 instanceof DropDatabaseMessage); + Assert.assertEquals("mydb", message2.getDB()); + } else if (event.equals(HCatConstants.HCAT_PARTITION_DONE_EVENT)) { + // TODO: Fill in when PARTITION_DONE_EVENT is supported. + Assert.assertTrue("Unexpected: HCAT_PARTITION_DONE_EVENT not supported (yet).", false); + } else { + Assert.assertTrue("Unexpected event-type: " + event, false); + } + + } catch (JMSException e) { + e.printStackTrace(System.err); + assert false; + } + } +} diff --git hcatalog/src/java/org/apache/hcatalog/package-info.java hcatalog/src/java/org/apache/hcatalog/package-info.java new file mode 100644 index 0000000..7faf206 --- /dev/null +++ hcatalog/src/java/org/apache/hcatalog/package-info.java @@ -0,0 +1,23 @@ +/** + * 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. + */ + +/** + * @deprecated classes in {@link org.apache.hive.hcatalog} should be used instead + */ +package org.apache.hcatalog; diff --git hcatalog/webhcat/java-client/src/main/java/org/apache/hcatalog/api/ConnectionFailureException.java hcatalog/webhcat/java-client/src/main/java/org/apache/hcatalog/api/ConnectionFailureException.java new file mode 100644 index 0000000..c5a8a0a --- /dev/null +++ hcatalog/webhcat/java-client/src/main/java/org/apache/hcatalog/api/ConnectionFailureException.java @@ -0,0 +1,40 @@ +/** + * 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.hcatalog.api; + +import org.apache.hcatalog.common.HCatException; +/** + * Class representing exceptions resulting from connection problems + * between HCat client and server. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.api.ConnectionFailureException} instead + */ +public class ConnectionFailureException extends HCatException { + + private static final long serialVersionUID = 1L; + + /** + * @param message Exception message. + * @param cause The wrapped Throwable that caused this exception. + */ + public ConnectionFailureException(String message, Throwable cause) { + super(message, cause); + } + +} diff --git hcatalog/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatAddPartitionDesc.java hcatalog/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatAddPartitionDesc.java new file mode 100644 index 0000000..562b2fc --- /dev/null +++ hcatalog/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatAddPartitionDesc.java @@ -0,0 +1,186 @@ +/** + * 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.hcatalog.api; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.metastore.MetaStoreUtils; +import org.apache.hadoop.hive.metastore.Warehouse; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hcatalog.common.HCatException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The Class HCatAddPartitionDesc helps users in defining partition attributes. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.api.HCatAddPartitionDesc} instead + */ +public class HCatAddPartitionDesc { + + private static final Logger LOG = LoggerFactory.getLogger(HCatAddPartitionDesc.class); + private String tableName; + private String dbName; + private String location; + private Map partSpec; + + private HCatAddPartitionDesc(String dbName, String tbl, String loc, Map spec) { + this.dbName = dbName; + this.tableName = tbl; + this.location = loc; + this.partSpec = spec; + } + + /** + * Gets the location. + * + * @return the location + */ + public String getLocation() { + return this.location; + } + + + /** + * Gets the partition spec. + * + * @return the partition spec + */ + public Map getPartitionSpec() { + return this.partSpec; + } + + /** + * Gets the table name. + * + * @return the table name + */ + public String getTableName() { + return this.tableName; + } + + /** + * Gets the database name. + * + * @return the database name + */ + public String getDatabaseName() { + return this.dbName; + } + + @Override + public String toString() { + return "HCatAddPartitionDesc [" + + (tableName != null ? "tableName=" + tableName + ", " : "tableName=null") + + (dbName != null ? "dbName=" + dbName + ", " : "dbName=null") + + (location != null ? "location=" + location + ", " : "location=null") + + (partSpec != null ? "partSpec=" + partSpec : "partSpec=null") + "]"; + } + + /** + * Creates the builder for specifying attributes. + * + * @param dbName the db name + * @param tableName the table name + * @param location the location + * @param partSpec the part spec + * @return the builder + * @throws HCatException + */ + public static Builder create(String dbName, String tableName, String location, + Map partSpec) throws HCatException { + return new Builder(dbName, tableName, location, partSpec); + } + + Partition toHivePartition(Table hiveTable) throws HCatException { + Partition hivePtn = new Partition(); + hivePtn.setDbName(this.dbName); + hivePtn.setTableName(this.tableName); + + List pvals = new ArrayList(); + for (FieldSchema field : hiveTable.getPartitionKeys()) { + String val = partSpec.get(field.getName()); + if (val == null || val.length() == 0) { + throw new HCatException("create partition: Value for key " + + field.getName() + " is null or empty"); + } + pvals.add(val); + } + + hivePtn.setValues(pvals); + StorageDescriptor sd = new StorageDescriptor(hiveTable.getSd()); + hivePtn.setSd(sd); + hivePtn.setParameters(hiveTable.getParameters()); + if (this.location != null) { + hivePtn.getSd().setLocation(this.location); + } else { + String partName; + try { + partName = Warehouse.makePartName( + hiveTable.getPartitionKeys(), pvals); + LOG.info("Setting partition location to :" + partName); + } catch (MetaException e) { + throw new HCatException("Exception while creating partition name.", e); + } + Path partPath = new Path(hiveTable.getSd().getLocation(), partName); + hivePtn.getSd().setLocation(partPath.toString()); + } + hivePtn.setCreateTime((int) (System.currentTimeMillis() / 1000)); + hivePtn.setLastAccessTimeIsSet(false); + return hivePtn; + } + + public static class Builder { + + private String tableName; + private String location; + private Map values; + private String dbName; + + private Builder(String dbName, String tableName, String location, Map values) { + this.dbName = dbName; + this.tableName = tableName; + this.location = location; + this.values = values; + } + + /** + * Builds the HCatAddPartitionDesc. + * + * @return the h cat add partition desc + * @throws HCatException + */ + public HCatAddPartitionDesc build() throws HCatException { + if (this.dbName == null) { + this.dbName = MetaStoreUtils.DEFAULT_DATABASE_NAME; + } + HCatAddPartitionDesc desc = new HCatAddPartitionDesc( + this.dbName, this.tableName, this.location, + this.values); + return desc; + } + } + +} diff --git hcatalog/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatClient.java hcatalog/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatClient.java new file mode 100644 index 0000000..33f4434 --- /dev/null +++ hcatalog/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatClient.java @@ -0,0 +1,362 @@ +/** + * 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.hcatalog.api; + +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.common.JavaUtils; +import org.apache.hadoop.hive.metastore.api.PartitionEventType; +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.data.schema.HCatFieldSchema; + +/** + * The abstract class HCatClient containing APIs for HCatalog DDL commands. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.api.HCatClient} instead + */ +public abstract class HCatClient { + + public enum DropDBMode {RESTRICT, CASCADE} + + public static final String HCAT_CLIENT_IMPL_CLASS = "hcat.client.impl.class"; + + /** + * Creates an instance of HCatClient. + * + * @param conf An instance of configuration. + * @return An instance of HCatClient. + * @throws HCatException + */ + public static HCatClient create(Configuration conf) throws HCatException { + HCatClient client = null; + String className = conf.get(HCAT_CLIENT_IMPL_CLASS, + HCatClientHMSImpl.class.getName()); + try { + Class clientClass = Class.forName(className, + true, JavaUtils.getClassLoader()).asSubclass( + HCatClient.class); + client = (HCatClient) clientClass.newInstance(); + } catch (ClassNotFoundException e) { + throw new HCatException( + "ClassNotFoundException while creating client class.", e); + } catch (InstantiationException e) { + throw new HCatException( + "InstantiationException while creating client class.", e); + } catch (IllegalAccessException e) { + throw new HCatException( + "IllegalAccessException while creating client class.", e); + } + if (client != null) { + client.initialize(conf); + } + return client; + } + + abstract void initialize(Configuration conf) throws HCatException; + + /** + * Get all existing databases that match the given + * pattern. The matching occurs as per Java regular expressions + * + * @param pattern java re pattern + * @return list of database names + * @throws HCatException + */ + public abstract List listDatabaseNamesByPattern(String pattern) + throws HCatException; + + /** + * Gets the database. + * + * @param dbName The name of the database. + * @return An instance of HCatDatabaseInfo. + * @throws HCatException + */ + public abstract HCatDatabase getDatabase(String dbName) throws HCatException; + + /** + * Creates the database. + * + * @param dbInfo An instance of HCatCreateDBDesc. + * @throws HCatException + */ + public abstract void createDatabase(HCatCreateDBDesc dbInfo) + throws HCatException; + + /** + * Drops a database. + * + * @param dbName The name of the database to delete. + * @param ifExists Hive returns an error if the database specified does not exist, + * unless ifExists is set to true. + * @param mode This is set to either "restrict" or "cascade". Restrict will + * remove the schema if all the tables are empty. Cascade removes + * everything including data and definitions. + * @throws HCatException + */ + public abstract void dropDatabase(String dbName, boolean ifExists, + DropDBMode mode) throws HCatException; + + /** + * Returns all existing tables from the specified database which match the given + * pattern. The matching occurs as per Java regular expressions. + * @param dbName The name of the DB (to be searched) + * @param tablePattern The regex for the table-name + * @return list of table names + * @throws HCatException + */ + public abstract List listTableNamesByPattern(String dbName, String tablePattern) + throws HCatException; + + /** + * Gets the table. + * + * @param dbName The name of the database. + * @param tableName The name of the table. + * @return An instance of HCatTableInfo. + * @throws HCatException + */ + public abstract HCatTable getTable(String dbName, String tableName) + throws HCatException; + + /** + * Creates the table. + * + * @param createTableDesc An instance of HCatCreateTableDesc class. + * @throws HCatException + */ + public abstract void createTable(HCatCreateTableDesc createTableDesc) throws HCatException; + + /** + * Updates the Table's column schema to the specified definition. + * + * @param dbName The name of the database. + * @param tableName The name of the table. + * @param columnSchema The (new) definition of the column schema (i.e. list of fields). + * + */ + public abstract void updateTableSchema(String dbName, String tableName, List columnSchema) + throws HCatException; + + /** + * Creates the table like an existing table. + * + * @param dbName The name of the database. + * @param existingTblName The name of the existing table. + * @param newTableName The name of the new table. + * @param ifNotExists If true, then error related to already table existing is skipped. + * @param isExternal Set to "true", if table has be created at a different + * location other than default. + * @param location The location for the table. + * @throws HCatException + */ + public abstract void createTableLike(String dbName, String existingTblName, + String newTableName, boolean ifNotExists, boolean isExternal, + String location) throws HCatException; + + /** + * Drop table. + * + * @param dbName The name of the database. + * @param tableName The name of the table. + * @param ifExists Hive returns an error if the database specified does not exist, + * unless ifExists is set to true. + * @throws HCatException + */ + public abstract void dropTable(String dbName, String tableName, + boolean ifExists) throws HCatException; + + /** + * Renames a table. + * + * @param dbName The name of the database. + * @param oldName The name of the table to be renamed. + * @param newName The new name of the table. + * @throws HCatException + */ + public abstract void renameTable(String dbName, String oldName, + String newName) throws HCatException; + + /** + * Gets all the partitions. + * + * @param dbName The name of the database. + * @param tblName The name of the table. + * @return A list of partitions. + * @throws HCatException + */ + public abstract List getPartitions(String dbName, String tblName) + throws HCatException; + + /** + * Gets all the partitions that match the specified (and possibly partial) partition specification. + * A partial partition-specification is one where not all partition-keys have associated values. For example, + * for a table ('myDb.myTable') with 2 partition keys (dt string, region string), + * if for each dt ('20120101', '20120102', etc.) there can exist 3 regions ('us', 'uk', 'in'), then, + * 1. Complete partition spec: getPartitions('myDb', 'myTable', {dt='20120101', region='us'}) would return 1 partition. + * 2. Partial partition spec: getPartitions('myDb', 'myTable', {dt='20120101'}) would return all 3 partitions, + * with dt='20120101' (i.e. region = 'us', 'uk' and 'in'). + * @param dbName The name of the database. + * @param tblName The name of the table. + * @param partitionSpec The partition specification. (Need not include all partition keys.) + * @return A list of partitions. + * @throws HCatException + */ + public abstract List getPartitions(String dbName, String tblName, Map partitionSpec) + throws HCatException; + + /** + * Gets the partition. + * + * @param dbName The database name. + * @param tableName The table name. + * @param partitionSpec The partition specification, {[col_name,value],[col_name2,value2]}. All partition-key-values + * must be specified. + * @return An instance of HCatPartitionInfo. + * @throws HCatException + */ + public abstract HCatPartition getPartition(String dbName, String tableName, + Map partitionSpec) throws HCatException; + + /** + * Adds the partition. + * + * @param partInfo An instance of HCatAddPartitionDesc. + * @throws HCatException + */ + public abstract void addPartition(HCatAddPartitionDesc partInfo) + throws HCatException; + + /** + * Adds a list of partitions. + * + * @param partInfoList A list of HCatAddPartitionDesc. + * @return The number of partitions added. + * @throws HCatException + */ + public abstract int addPartitions(List partInfoList) + throws HCatException; + + /** + * Drops partition(s) that match the specified (and possibly partial) partition specification. + * A partial partition-specification is one where not all partition-keys have associated values. For example, + * for a table ('myDb.myTable') with 2 partition keys (dt string, region string), + * if for each dt ('20120101', '20120102', etc.) there can exist 3 regions ('us', 'uk', 'in'), then, + * 1. Complete partition spec: dropPartitions('myDb', 'myTable', {dt='20120101', region='us'}) would drop 1 partition. + * 2. Partial partition spec: dropPartitions('myDb', 'myTable', {dt='20120101'}) would drop all 3 partitions, + * with dt='20120101' (i.e. region = 'us', 'uk' and 'in'). + * @param dbName The database name. + * @param tableName The table name. + * @param partitionSpec The partition specification, {[col_name,value],[col_name2,value2]}. + * @param ifExists Hive returns an error if the partition specified does not exist, unless ifExists is set to true. + * @throws HCatException,ConnectionFailureException + */ + public abstract void dropPartitions(String dbName, String tableName, + Map partitionSpec, boolean ifExists) + throws HCatException; + + /** + * List partitions by filter. + * + * @param dbName The database name. + * @param tblName The table name. + * @param filter The filter string, + * for example "part1 = \"p1_abc\" and part2 <= "\p2_test\"". Filtering can + * be done only on string partition keys. + * @return list of partitions + * @throws HCatException + */ + public abstract List listPartitionsByFilter(String dbName, String tblName, + String filter) throws HCatException; + + /** + * Mark partition for event. + * + * @param dbName The database name. + * @param tblName The table name. + * @param partKVs the key-values associated with the partition. + * @param eventType the event type + * @throws HCatException + */ + public abstract void markPartitionForEvent(String dbName, String tblName, + Map partKVs, PartitionEventType eventType) + throws HCatException; + + /** + * Checks if a partition is marked for event. + * + * @param dbName the db name + * @param tblName the table name + * @param partKVs the key-values associated with the partition. + * @param eventType the event type + * @return true, if is partition marked for event + * @throws HCatException + */ + public abstract boolean isPartitionMarkedForEvent(String dbName, String tblName, + Map partKVs, PartitionEventType eventType) + throws HCatException; + + /** + * Gets the delegation token. + * + * @param owner the owner + * @param renewerKerberosPrincipalName the renewer kerberos principal name + * @return the delegation token + * @throws HCatException,ConnectionFailureException + */ + public abstract String getDelegationToken(String owner, + String renewerKerberosPrincipalName) throws HCatException; + + /** + * Renew delegation token. + * + * @param tokenStrForm the token string + * @return the new expiration time + * @throws HCatException + */ + public abstract long renewDelegationToken(String tokenStrForm) + throws HCatException; + + /** + * Cancel delegation token. + * + * @param tokenStrForm the token string + * @throws HCatException + */ + public abstract void cancelDelegationToken(String tokenStrForm) + throws HCatException; + + /** + * Retrieve Message-bus topic for a table. + * + * @param dbName The name of the DB. + * @param tableName The name of the table. + * @return Topic-name for the message-bus on which messages will be sent for the specified table. + * By default, this is set to .. Returns null when not set. + */ + public abstract String getMessageBusTopicName(String dbName, String tableName) throws HCatException; + + /** + * Close the hcatalog client. + * + * @throws HCatException + */ + public abstract void close() throws HCatException; +} diff --git hcatalog/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatClientHMSImpl.java hcatalog/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatClientHMSImpl.java new file mode 100644 index 0000000..0dbb47f --- /dev/null +++ hcatalog/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatClientHMSImpl.java @@ -0,0 +1,724 @@ +/** + * 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.hcatalog.api; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.MetaStoreUtils; +import org.apache.hadoop.hive.metastore.TableType; +import org.apache.hadoop.hive.metastore.api.AlreadyExistsException; +import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.hadoop.hive.metastore.api.InvalidObjectException; +import org.apache.hadoop.hive.metastore.api.InvalidOperationException; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.PartitionEventType; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.metastore.api.UnknownDBException; +import org.apache.hadoop.hive.metastore.api.UnknownTableException; +import org.apache.hcatalog.common.HCatConstants; +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.common.HCatUtil; +import org.apache.hcatalog.data.schema.HCatFieldSchema; +import org.apache.hcatalog.data.schema.HCatSchemaUtils; +import org.apache.thrift.TException; + +/** + * The HCatClientHMSImpl is the Hive Metastore client based implementation of + * HCatClient. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.api.HCatClientHMSImpl} instead + */ +public class HCatClientHMSImpl extends HCatClient { + + private HiveMetaStoreClient hmsClient; + private Configuration config; + private HiveConf hiveConfig; + + @Override + public List listDatabaseNamesByPattern(String pattern) + throws HCatException { + List dbNames = null; + try { + dbNames = hmsClient.getDatabases(pattern); + } catch (MetaException exp) { + throw new HCatException("MetaException while listing db names", exp); + } + return dbNames; + } + + @Override + public HCatDatabase getDatabase(String dbName) throws HCatException { + HCatDatabase db = null; + try { + Database hiveDB = hmsClient.getDatabase(checkDB(dbName)); + if (hiveDB != null) { + db = new HCatDatabase(hiveDB); + } + } catch (NoSuchObjectException exp) { + throw new ObjectNotFoundException( + "NoSuchObjectException while fetching database", exp); + } catch (MetaException exp) { + throw new HCatException("MetaException while fetching database", + exp); + } catch (TException exp) { + throw new ConnectionFailureException( + "TException while fetching database", exp); + } + return db; + } + + @Override + public void createDatabase(HCatCreateDBDesc dbInfo) throws HCatException { + try { + hmsClient.createDatabase(dbInfo.toHiveDb()); + } catch (AlreadyExistsException exp) { + if (!dbInfo.getIfNotExists()) { + throw new HCatException( + "AlreadyExistsException while creating database", exp); + } + } catch (InvalidObjectException exp) { + throw new HCatException( + "InvalidObjectException while creating database", exp); + } catch (MetaException exp) { + throw new HCatException("MetaException while creating database", + exp); + } catch (TException exp) { + throw new ConnectionFailureException( + "TException while creating database", exp); + } + } + + @Override + public void dropDatabase(String dbName, boolean ifExists, DropDBMode mode) + throws HCatException { + boolean isCascade = mode.toString().equalsIgnoreCase("cascade"); + try { + hmsClient.dropDatabase(checkDB(dbName), true, ifExists, isCascade); + } catch (NoSuchObjectException e) { + if (!ifExists) { + throw new ObjectNotFoundException( + "NoSuchObjectException while dropping db.", e); + } + } catch (InvalidOperationException e) { + throw new HCatException( + "InvalidOperationException while dropping db.", e); + } catch (MetaException e) { + throw new HCatException("MetaException while dropping db.", e); + } catch (TException e) { + throw new ConnectionFailureException("TException while dropping db.", + e); + } + } + + @Override + public List listTableNamesByPattern(String dbName, + String tablePattern) throws HCatException { + List tableNames = null; + try { + tableNames = hmsClient.getTables(checkDB(dbName), tablePattern); + } catch (MetaException e) { + throw new HCatException( + "MetaException while fetching table names.", e); + } + return tableNames; + } + + @Override + public HCatTable getTable(String dbName, String tableName) + throws HCatException { + HCatTable table = null; + try { + Table hiveTable = hmsClient.getTable(checkDB(dbName), tableName); + if (hiveTable != null) { + table = new HCatTable(hiveTable); + } + } catch (MetaException e) { + throw new HCatException("MetaException while fetching table.", e); + } catch (NoSuchObjectException e) { + throw new ObjectNotFoundException( + "NoSuchObjectException while fetching table.", e); + } catch (TException e) { + throw new ConnectionFailureException( + "TException while fetching table.", e); + } + return table; + } + + @Override + public void createTable(HCatCreateTableDesc createTableDesc) + throws HCatException { + try { + hmsClient.createTable(createTableDesc.toHiveTable(hiveConfig)); + } catch (AlreadyExistsException e) { + if (!createTableDesc.getIfNotExists()) { + throw new HCatException( + "AlreadyExistsException while creating table.", e); + } + } catch (InvalidObjectException e) { + throw new HCatException( + "InvalidObjectException while creating table.", e); + } catch (MetaException e) { + throw new HCatException("MetaException while creating table.", e); + } catch (NoSuchObjectException e) { + throw new ObjectNotFoundException( + "NoSuchObjectException while creating table.", e); + } catch (TException e) { + throw new ConnectionFailureException( + "TException while creating table.", e); + } catch (IOException e) { + throw new HCatException("IOException while creating hive conf.", e); + } + + } + + @Override + public void updateTableSchema(String dbName, String tableName, List columnSchema) + throws HCatException { + try { + Table table = hmsClient.getTable(dbName, tableName); + table.getSd().setCols(HCatSchemaUtils.getFieldSchemas(columnSchema)); + hmsClient.alter_table(dbName, tableName, table); + } + catch (InvalidOperationException e) { + throw new HCatException("InvalidOperationException while updating table schema.", e); + } + catch (MetaException e) { + throw new HCatException("MetaException while updating table schema.", e); + } + catch (NoSuchObjectException e) { + throw new ObjectNotFoundException( + "NoSuchObjectException while updating table schema.", e); + } + catch (TException e) { + throw new ConnectionFailureException( + "TException while updating table schema.", e); + } + } + + @Override + public void createTableLike(String dbName, String existingTblName, + String newTableName, boolean ifNotExists, boolean isExternal, + String location) throws HCatException { + + Table hiveTable = getHiveTableLike(checkDB(dbName), existingTblName, + newTableName, ifNotExists, location); + if (hiveTable != null) { + try { + hmsClient.createTable(hiveTable); + } catch (AlreadyExistsException e) { + if (!ifNotExists) { + throw new HCatException( + "A table already exists with the name " + + newTableName, e); + } + } catch (InvalidObjectException e) { + throw new HCatException( + "InvalidObjectException in create table like command.", + e); + } catch (MetaException e) { + throw new HCatException( + "MetaException in create table like command.", e); + } catch (NoSuchObjectException e) { + throw new ObjectNotFoundException( + "NoSuchObjectException in create table like command.", + e); + } catch (TException e) { + throw new ConnectionFailureException( + "TException in create table like command.", e); + } + } + } + + @Override + public void dropTable(String dbName, String tableName, boolean ifExists) + throws HCatException { + try { + hmsClient.dropTable(checkDB(dbName), tableName, true, ifExists); + } catch (NoSuchObjectException e) { + if (!ifExists) { + throw new ObjectNotFoundException( + "NoSuchObjectException while dropping table.", e); + } + } catch (MetaException e) { + throw new HCatException("MetaException while dropping table.", e); + } catch (TException e) { + throw new ConnectionFailureException( + "TException while dropping table.", e); + } + } + + @Override + public void renameTable(String dbName, String oldName, String newName) + throws HCatException { + Table tbl; + try { + Table oldtbl = hmsClient.getTable(checkDB(dbName), oldName); + if (oldtbl != null) { + // TODO : Should be moved out. + if (oldtbl + .getParameters() + .get(org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_STORAGE) != null) { + throw new HCatException( + "Cannot use rename command on a non-native table"); + } + tbl = new Table(oldtbl); + tbl.setTableName(newName); + hmsClient.alter_table(checkDB(dbName), oldName, tbl); + } + } catch (MetaException e) { + throw new HCatException("MetaException while renaming table", e); + } catch (NoSuchObjectException e) { + throw new ObjectNotFoundException( + "NoSuchObjectException while renaming table", e); + } catch (InvalidOperationException e) { + throw new HCatException( + "InvalidOperationException while renaming table", e); + } catch (TException e) { + throw new ConnectionFailureException( + "TException while renaming table", e); + } + } + + @Override + public List getPartitions(String dbName, String tblName) + throws HCatException { + List hcatPtns = new ArrayList(); + try { + List hivePtns = hmsClient.listPartitions( + checkDB(dbName), tblName, (short) -1); + for (Partition ptn : hivePtns) { + hcatPtns.add(new HCatPartition(ptn)); + } + } catch (NoSuchObjectException e) { + throw new ObjectNotFoundException( + "NoSuchObjectException while retrieving partition.", e); + } catch (MetaException e) { + throw new HCatException( + "MetaException while retrieving partition.", e); + } catch (TException e) { + throw new ConnectionFailureException( + "TException while retrieving partition.", e); + } + return hcatPtns; + } + + @Override + public List getPartitions(String dbName, String tblName, Map partitionSpec) throws HCatException { + return listPartitionsByFilter(dbName, tblName, getFilterString(partitionSpec)); + } + + private static String getFilterString(Map partitionSpec) { + final String AND = " AND "; + + StringBuilder filter = new StringBuilder(); + for (Map.Entry entry : partitionSpec.entrySet()) { + filter.append(entry.getKey()).append("=").append("\"").append(entry.getValue()).append("\"").append(AND); + } + + int length = filter.toString().length(); + if (length > 0) + filter.delete(length - AND.length(), length); + + return filter.toString(); + } + + @Override + public HCatPartition getPartition(String dbName, String tableName, + Map partitionSpec) throws HCatException { + HCatPartition partition = null; + try { + List partitionColumns = getTable(checkDB(dbName), tableName).getPartCols(); + if (partitionColumns.size() != partitionSpec.size()) { + throw new HCatException("Partition-spec doesn't have the right number of partition keys."); + } + + ArrayList ptnValues = new ArrayList(); + for (HCatFieldSchema partitionColumn : partitionColumns) { + String partKey = partitionColumn.getName(); + if (partitionSpec.containsKey(partKey)) { + ptnValues.add(partitionSpec.get(partKey)); // Partition-keys added in order. + } + else { + throw new HCatException("Invalid partition-key specified: " + partKey); + } + } + Partition hivePartition = hmsClient.getPartition(checkDB(dbName), + tableName, ptnValues); + if (hivePartition != null) { + partition = new HCatPartition(hivePartition); + } + } catch (MetaException e) { + throw new HCatException( + "MetaException while retrieving partition.", e); + } catch (NoSuchObjectException e) { + throw new ObjectNotFoundException( + "NoSuchObjectException while retrieving partition.", e); + } catch (TException e) { + throw new ConnectionFailureException( + "TException while retrieving partition.", e); + } + return partition; + } + + @Override + public void addPartition(HCatAddPartitionDesc partInfo) + throws HCatException { + Table tbl = null; + try { + tbl = hmsClient.getTable(partInfo.getDatabaseName(), + partInfo.getTableName()); + // TODO: Should be moved out. + if (tbl.getPartitionKeysSize() == 0) { + throw new HCatException("The table " + partInfo.getTableName() + + " is not partitioned."); + } + + hmsClient.add_partition(partInfo.toHivePartition(tbl)); + } catch (InvalidObjectException e) { + throw new HCatException( + "InvalidObjectException while adding partition.", e); + } catch (AlreadyExistsException e) { + throw new HCatException( + "AlreadyExistsException while adding partition.", e); + } catch (MetaException e) { + throw new HCatException("MetaException while adding partition.", e); + } catch (NoSuchObjectException e) { + throw new ObjectNotFoundException("The table " + partInfo.getTableName() + + " is could not be found.", e); + } catch (TException e) { + throw new ConnectionFailureException( + "TException while adding partition.", e); + } + } + + @Override + public void dropPartitions(String dbName, String tableName, + Map partitionSpec, boolean ifExists) + throws HCatException { + try { + dbName = checkDB(dbName); + List partitions = hmsClient.listPartitionsByFilter(dbName, tableName, + getFilterString(partitionSpec), (short)-1); + + for (Partition partition : partitions) { + dropPartition(partition, ifExists); + } + + } catch (NoSuchObjectException e) { + throw new ObjectNotFoundException( + "NoSuchObjectException while dropping partition. " + + "Either db(" + dbName + ") or table(" + tableName + ") missing.", e); + } catch (MetaException e) { + throw new HCatException("MetaException while dropping partition.", + e); + } catch (TException e) { + throw new ConnectionFailureException( + "TException while dropping partition.", e); + } + } + + private void dropPartition(Partition partition, boolean ifExists) + throws HCatException, MetaException, TException { + try { + hmsClient.dropPartition(partition.getDbName(), partition.getTableName(), partition.getValues()); + } catch (NoSuchObjectException e) { + if (!ifExists) { + throw new ObjectNotFoundException( + "NoSuchObjectException while dropping partition: " + partition.getValues(), e); + } + } + } + + @Override + public List listPartitionsByFilter(String dbName, + String tblName, String filter) throws HCatException { + List hcatPtns = new ArrayList(); + try { + List hivePtns = hmsClient.listPartitionsByFilter( + checkDB(dbName), tblName, filter, (short) -1); + for (Partition ptn : hivePtns) { + hcatPtns.add(new HCatPartition(ptn)); + } + } catch (MetaException e) { + throw new HCatException("MetaException while fetching partitions.", + e); + } catch (NoSuchObjectException e) { + throw new ObjectNotFoundException( + "NoSuchObjectException while fetching partitions.", e); + } catch (TException e) { + throw new ConnectionFailureException( + "TException while fetching partitions.", e); + } + return hcatPtns; + } + + @Override + public void markPartitionForEvent(String dbName, String tblName, + Map partKVs, PartitionEventType eventType) + throws HCatException { + try { + hmsClient.markPartitionForEvent(checkDB(dbName), tblName, partKVs, + eventType); + } catch (MetaException e) { + throw new HCatException( + "MetaException while marking partition for event.", e); + } catch (NoSuchObjectException e) { + throw new ObjectNotFoundException( + "NoSuchObjectException while marking partition for event.", + e); + } catch (UnknownTableException e) { + throw new HCatException( + "UnknownTableException while marking partition for event.", + e); + } catch (UnknownDBException e) { + throw new HCatException( + "UnknownDBException while marking partition for event.", e); + } catch (TException e) { + throw new ConnectionFailureException( + "TException while marking partition for event.", e); + } + } + + @Override + public boolean isPartitionMarkedForEvent(String dbName, String tblName, + Map partKVs, PartitionEventType eventType) + throws HCatException { + boolean isMarked = false; + try { + isMarked = hmsClient.isPartitionMarkedForEvent(checkDB(dbName), + tblName, partKVs, eventType); + } catch (MetaException e) { + throw new HCatException( + "MetaException while checking partition for event.", e); + } catch (NoSuchObjectException e) { + throw new ObjectNotFoundException( + "NoSuchObjectException while checking partition for event.", + e); + } catch (UnknownTableException e) { + throw new HCatException( + "UnknownTableException while checking partition for event.", + e); + } catch (UnknownDBException e) { + throw new HCatException( + "UnknownDBException while checking partition for event.", e); + } catch (TException e) { + throw new ConnectionFailureException( + "TException while checking partition for event.", e); + } + return isMarked; + } + + @Override + public String getDelegationToken(String owner, + String renewerKerberosPrincipalName) throws HCatException { + String token = null; + try { + token = hmsClient.getDelegationToken(owner, + renewerKerberosPrincipalName); + } catch (MetaException e) { + throw new HCatException( + "MetaException while getting delegation token.", e); + } catch (TException e) { + throw new ConnectionFailureException( + "TException while getting delegation token.", e); + } + + return token; + } + + @Override + public long renewDelegationToken(String tokenStrForm) throws HCatException { + long time = 0; + try { + time = hmsClient.renewDelegationToken(tokenStrForm); + } catch (MetaException e) { + throw new HCatException( + "MetaException while renewing delegation token.", e); + } catch (TException e) { + throw new ConnectionFailureException( + "TException while renewing delegation token.", e); + } + + return time; + } + + @Override + public void cancelDelegationToken(String tokenStrForm) + throws HCatException { + try { + hmsClient.cancelDelegationToken(tokenStrForm); + } catch (MetaException e) { + throw new HCatException( + "MetaException while canceling delegation token.", e); + } catch (TException e) { + throw new ConnectionFailureException( + "TException while canceling delegation token.", e); + } + } + + /* + * @param conf /* @throws HCatException,ConnectionFailureException + * + * @see + * org.apache.hcatalog.api.HCatClient#initialize(org.apache.hadoop.conf. + * Configuration) + */ + @Override + void initialize(Configuration conf) throws HCatException { + this.config = conf; + try { + hiveConfig = HCatUtil.getHiveConf(config); + hmsClient = HCatUtil.getHiveClient(hiveConfig); + } catch (MetaException exp) { + throw new HCatException("MetaException while creating HMS client", + exp); + } catch (IOException exp) { + throw new HCatException("IOException while creating HMS client", + exp); + } + + } + + private Table getHiveTableLike(String dbName, String existingTblName, + String newTableName, boolean isExternal, String location) + throws HCatException { + Table oldtbl = null; + Table newTable = null; + try { + oldtbl = hmsClient.getTable(checkDB(dbName), existingTblName); + } catch (MetaException e1) { + throw new HCatException( + "MetaException while retrieving existing table.", e1); + } catch (NoSuchObjectException e1) { + throw new ObjectNotFoundException( + "NoSuchObjectException while retrieving existing table.", + e1); + } catch (TException e1) { + throw new ConnectionFailureException( + "TException while retrieving existing table.", e1); + } + if (oldtbl != null) { + newTable = new Table(); + newTable.setTableName(newTableName); + newTable.setDbName(dbName); + StorageDescriptor sd = new StorageDescriptor(oldtbl.getSd()); + newTable.setSd(sd); + newTable.setParameters(oldtbl.getParameters()); + if (location == null) { + newTable.getSd().setLocation(oldtbl.getSd().getLocation()); + } else { + newTable.getSd().setLocation(location); + } + if (isExternal) { + newTable.putToParameters("EXTERNAL", "TRUE"); + newTable.setTableType(TableType.EXTERNAL_TABLE.toString()); + } else { + newTable.getParameters().remove("EXTERNAL"); + } + // set create time + newTable.setCreateTime((int) (System.currentTimeMillis() / 1000)); + newTable.setLastAccessTimeIsSet(false); + } + return newTable; + } + + /* + * @throws HCatException + * + * @see org.apache.hcatalog.api.HCatClient#closeClient() + */ + @Override + public void close() throws HCatException { + hmsClient.close(); + } + + private String checkDB(String name) { + if (StringUtils.isEmpty(name)) { + return MetaStoreUtils.DEFAULT_DATABASE_NAME; + } else { + return name; + } + } + + /* + * @param partInfoList + * @return The size of the list of partitions. + * @throws HCatException,ConnectionFailureException + * @see org.apache.hcatalog.api.HCatClient#addPartitions(java.util.List) + */ + @Override + public int addPartitions(List partInfoList) + throws HCatException { + int numPartitions = -1; + if ((partInfoList == null) || (partInfoList.size() == 0)) { + throw new HCatException("The partition list is null or empty."); + } + + Table tbl = null; + try { + tbl = hmsClient.getTable(partInfoList.get(0).getDatabaseName(), + partInfoList.get(0).getTableName()); + ArrayList ptnList = new ArrayList(); + for (HCatAddPartitionDesc desc : partInfoList) { + ptnList.add(desc.toHivePartition(tbl)); + } + numPartitions = hmsClient.add_partitions(ptnList); + } catch (InvalidObjectException e) { + throw new HCatException( + "InvalidObjectException while adding partition.", e); + } catch (AlreadyExistsException e) { + throw new HCatException( + "AlreadyExistsException while adding partition.", e); + } catch (MetaException e) { + throw new HCatException("MetaException while adding partition.", e); + } catch (NoSuchObjectException e) { + throw new ObjectNotFoundException("The table " + + partInfoList.get(0).getTableName() + + " is could not be found.", e); + } catch (TException e) { + throw new ConnectionFailureException( + "TException while adding partition.", e); + } + return numPartitions; + } + + @Override + public String getMessageBusTopicName(String dbName, String tableName) throws HCatException { + try { + return hmsClient.getTable(dbName, tableName).getParameters().get(HCatConstants.HCAT_MSGBUS_TOPIC_NAME); + } + catch (MetaException e) { + throw new HCatException("MetaException while retrieving JMS Topic name.", e); + } catch (NoSuchObjectException e) { + throw new HCatException("Could not find DB:" + dbName + " or Table:" + tableName, e); + } catch (TException e) { + throw new ConnectionFailureException( + "TException while retrieving JMS Topic name.", e); + } + } +} diff --git hcatalog/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatCreateDBDesc.java hcatalog/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatCreateDBDesc.java new file mode 100644 index 0000000..25080ac --- /dev/null +++ hcatalog/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatCreateDBDesc.java @@ -0,0 +1,195 @@ +/** + * 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.hcatalog.api; + +import java.util.Map; + +import org.apache.hadoop.hive.metastore.api.Database; +import org.apache.hcatalog.common.HCatException; + +/** + * The Class HCatCreateDBDesc for defining database attributes. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.api.HCatCreateDBDesc} instead + */ +public class HCatCreateDBDesc { + + private String dbName; + private String locationUri; + private String comment; + private Map dbProperties; + private boolean ifNotExits = false; + + /** + * Gets the database properties. + * + * @return the database properties + */ + public Map getDatabaseProperties() { + return this.dbProperties; + } + + /** + * Gets the if not exists. + * + * @return the if not exists + */ + public boolean getIfNotExists() { + return this.ifNotExits; + } + + /** + * Gets the comments. + * + * @return the comments + */ + public String getComments() { + return this.comment; + } + + /** + * Gets the location. + * + * @return the location + */ + public String getLocation() { + return this.locationUri; + } + + /** + * Gets the database name. + * + * @return the database name + */ + public String getDatabaseName() { + return this.dbName; + } + + private HCatCreateDBDesc(String dbName) { + this.dbName = dbName; + } + + @Override + public String toString() { + return "HCatCreateDBDesc [" + + (dbName != null ? "dbName=" + dbName + ", " : "dbName=null") + + (locationUri != null ? "location=" + locationUri + ", " + : "location=null") + + (comment != null ? "comment=" + comment + ", " : "comment=null") + + (dbProperties != null ? "dbProperties=" + dbProperties + ", " + : "dbProperties=null") + "ifNotExits=" + ifNotExits + "]"; + } + + /** + * Creates the builder for defining attributes. + * + * @param dbName the db name + * @return the builder + */ + public static Builder create(String dbName) { + return new Builder(dbName); + } + + Database toHiveDb() { + Database hiveDB = new Database(); + hiveDB.setDescription(this.comment); + hiveDB.setLocationUri(this.locationUri); + hiveDB.setName(this.dbName); + hiveDB.setParameters(this.dbProperties); + return hiveDB; + } + + public static class Builder { + + private String innerLoc; + private String innerComment; + private Map innerDBProps; + private String dbName; + private boolean ifNotExists = false; + + private Builder(String dbName) { + this.dbName = dbName; + } + + /** + * Location. + * + * @param value the location of the database. + * @return the builder + */ + public Builder location(String value) { + this.innerLoc = value; + return this; + } + + /** + * Comment. + * + * @param value comments. + * @return the builder + */ + public Builder comment(String value) { + this.innerComment = value; + return this; + } + + /** + * If not exists. + * @param ifNotExists If set to true, hive will not throw exception, if a + * database with the same name already exists. + * @return the builder + */ + public Builder ifNotExists(boolean ifNotExists) { + this.ifNotExists = ifNotExists; + return this; + } + + /** + * Database properties. + * + * @param dbProps the database properties + * @return the builder + */ + public Builder databaseProperties(Map dbProps) { + this.innerDBProps = dbProps; + return this; + } + + + /** + * Builds the create database descriptor. + * + * @return An instance of HCatCreateDBDesc + * @throws HCatException + */ + public HCatCreateDBDesc build() throws HCatException { + if (this.dbName == null) { + throw new HCatException("Database name cannot be null."); + } + HCatCreateDBDesc desc = new HCatCreateDBDesc(this.dbName); + desc.comment = this.innerComment; + desc.locationUri = this.innerLoc; + desc.dbProperties = this.innerDBProps; + desc.ifNotExits = this.ifNotExists; + return desc; + + } + + } + +} diff --git hcatalog/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatCreateTableDesc.java hcatalog/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatCreateTableDesc.java new file mode 100644 index 0000000..542f7a5 --- /dev/null +++ hcatalog/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatCreateTableDesc.java @@ -0,0 +1,521 @@ +/** + * 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.hcatalog.api; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import org.apache.commons.lang.StringUtils; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.MetaStoreUtils; +import org.apache.hadoop.hive.metastore.TableType; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.Order; +import org.apache.hadoop.hive.metastore.api.SerDeInfo; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat; +import org.apache.hadoop.hive.ql.io.RCFileInputFormat; +import org.apache.hadoop.hive.ql.io.RCFileOutputFormat; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.metadata.HiveStorageHandler; +import org.apache.hadoop.hive.ql.metadata.HiveUtils; +import org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe; +import org.apache.hadoop.mapred.SequenceFileInputFormat; +import org.apache.hadoop.mapred.SequenceFileOutputFormat; +import org.apache.hadoop.mapred.TextInputFormat; +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.data.schema.HCatFieldSchema; +import org.apache.hcatalog.data.schema.HCatSchemaUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The Class HCatCreateTableDesc for defining attributes for a new table. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.api.HCatCreateTableDesc} instead + */ +@SuppressWarnings("deprecation") +public class HCatCreateTableDesc { + + private static final Logger LOG = LoggerFactory.getLogger(HCatCreateTableDesc.class); + + private String tableName; + private String dbName; + private boolean isExternal; + private String comment; + private String location; + private List cols; + private List partCols; + private List bucketCols; + private int numBuckets; + private List sortCols; + private Map tblProps; + private boolean ifNotExists; + private String fileFormat; + private String inputformat; + private String outputformat; + private String serde; + private String storageHandler; + + private HCatCreateTableDesc(String dbName, String tableName, List columns) { + this.dbName = dbName; + this.tableName = tableName; + this.cols = columns; + } + + /** + * Creates a builder for defining attributes. + * + * @param dbName the db name + * @param tableName the table name + * @param columns the columns + * @return the builder + */ + public static Builder create(String dbName, String tableName, List columns) { + return new Builder(dbName, tableName, columns); + } + + Table toHiveTable(HiveConf conf) throws HCatException { + + Table newTable = new Table(); + newTable.setDbName(dbName); + newTable.setTableName(tableName); + if (tblProps != null) { + newTable.setParameters(tblProps); + } + + if (isExternal) { + newTable.putToParameters("EXTERNAL", "TRUE"); + newTable.setTableType(TableType.EXTERNAL_TABLE.toString()); + } else { + newTable.setTableType(TableType.MANAGED_TABLE.toString()); + } + + StorageDescriptor sd = new StorageDescriptor(); + sd.setSerdeInfo(new SerDeInfo()); + if (location != null) { + sd.setLocation(location); + } + if (this.comment != null) { + newTable.putToParameters("comment", comment); + } + if (!StringUtils.isEmpty(fileFormat)) { + sd.setInputFormat(inputformat); + sd.setOutputFormat(outputformat); + if (serde != null) { + sd.getSerdeInfo().setSerializationLib(serde); + } else { + LOG.info("Using LazySimpleSerDe for table " + tableName); + sd.getSerdeInfo() + .setSerializationLib( + org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe.class + .getName()); + } + } else { + try { + LOG.info("Creating instance of storage handler to get input/output, serder info."); + HiveStorageHandler sh = HiveUtils.getStorageHandler(conf, + storageHandler); + sd.setInputFormat(sh.getInputFormatClass().getName()); + sd.setOutputFormat(sh.getOutputFormatClass().getName()); + sd.getSerdeInfo().setSerializationLib( + sh.getSerDeClass().getName()); + newTable.putToParameters( + org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_STORAGE, + storageHandler); + } catch (HiveException e) { + throw new HCatException( + "Exception while creating instance of storage handler", + e); + } + } + newTable.setSd(sd); + if (this.partCols != null) { + ArrayList hivePtnCols = new ArrayList(); + for (HCatFieldSchema fs : this.partCols) { + hivePtnCols.add(HCatSchemaUtils.getFieldSchema(fs)); + } + newTable.setPartitionKeys(hivePtnCols); + } + + if (this.cols != null) { + ArrayList hiveTblCols = new ArrayList(); + for (HCatFieldSchema fs : this.cols) { + hiveTblCols.add(HCatSchemaUtils.getFieldSchema(fs)); + } + newTable.getSd().setCols(hiveTblCols); + } + + if (this.bucketCols != null) { + newTable.getSd().setBucketCols(bucketCols); + newTable.getSd().setNumBuckets(numBuckets); + } + + if (this.sortCols != null) { + newTable.getSd().setSortCols(sortCols); + } + + newTable.setCreateTime((int) (System.currentTimeMillis() / 1000)); + newTable.setLastAccessTimeIsSet(false); + return newTable; + } + + /** + * Gets the if not exists. + * + * @return the if not exists + */ + public boolean getIfNotExists() { + return this.ifNotExists; + } + + /** + * Gets the table name. + * + * @return the table name + */ + public String getTableName() { + return this.tableName; + } + + /** + * Gets the cols. + * + * @return the cols + */ + public List getCols() { + return this.cols; + } + + /** + * Gets the partition cols. + * + * @return the partition cols + */ + public List getPartitionCols() { + return this.partCols; + } + + /** + * Gets the bucket cols. + * + * @return the bucket cols + */ + public List getBucketCols() { + return this.bucketCols; + } + + public int getNumBuckets() { + return this.numBuckets; + } + + /** + * Gets the comments. + * + * @return the comments + */ + public String getComments() { + return this.comment; + } + + /** + * Gets the storage handler. + * + * @return the storage handler + */ + public String getStorageHandler() { + return this.storageHandler; + } + + /** + * Gets the location. + * + * @return the location + */ + public String getLocation() { + return this.location; + } + + /** + * Gets the external. + * + * @return the external + */ + public boolean getExternal() { + return this.isExternal; + } + + /** + * Gets the sort cols. + * + * @return the sort cols + */ + public List getSortCols() { + return this.sortCols; + } + + /** + * Gets the tbl props. + * + * @return the tbl props + */ + public Map getTblProps() { + return this.tblProps; + } + + /** + * Gets the file format. + * + * @return the file format + */ + public String getFileFormat() { + return this.fileFormat; + } + + /** + * Gets the database name. + * + * @return the database name + */ + public String getDatabaseName() { + return this.dbName; + } + + @Override + public String toString() { + return "HCatCreateTableDesc [" + + (tableName != null ? "tableName=" + tableName + ", " : "tableName=null") + + (dbName != null ? "dbName=" + dbName + ", " : "dbName=null") + + "isExternal=" + + isExternal + + ", " + + (comment != null ? "comment=" + comment + ", " : "comment=null") + + (location != null ? "location=" + location + ", " : "location=null") + + (cols != null ? "cols=" + cols + ", " : "cols=null") + + (partCols != null ? "partCols=" + partCols + ", " : "partCols=null") + + (bucketCols != null ? "bucketCols=" + bucketCols + ", " : "bucketCols=null") + + "numBuckets=" + + numBuckets + + ", " + + (sortCols != null ? "sortCols=" + sortCols + ", " : "sortCols=null") + + (tblProps != null ? "tblProps=" + tblProps + ", " : "tblProps=null") + + "ifNotExists=" + + ifNotExists + + ", " + + (fileFormat != null ? "fileFormat=" + fileFormat + ", " : "fileFormat=null") + + (inputformat != null ? "inputformat=" + inputformat + ", " + : "inputformat=null") + + (outputformat != null ? "outputformat=" + outputformat + ", " + : "outputformat=null") + + (serde != null ? "serde=" + serde + ", " : "serde=null") + + (storageHandler != null ? "storageHandler=" + storageHandler + : "storageHandler=null") + "]"; + } + + public static class Builder { + + private String tableName; + private boolean isExternal; + private List cols; + private List partCols; + private List bucketCols; + private List sortCols; + private int numBuckets; + private String comment; + private String fileFormat; + private String location; + private String storageHandler; + private Map tblProps; + private boolean ifNotExists; + private String dbName; + + + private Builder(String dbName, String tableName, List columns) { + this.dbName = dbName; + this.tableName = tableName; + this.cols = columns; + } + + + /** + * If not exists. + * + * @param ifNotExists If set to true, hive will not throw exception, if a + * table with the same name already exists. + * @return the builder + */ + public Builder ifNotExists(boolean ifNotExists) { + this.ifNotExists = ifNotExists; + return this; + } + + + /** + * Partition cols. + * + * @param partCols the partition cols + * @return the builder + */ + public Builder partCols(List partCols) { + this.partCols = partCols; + return this; + } + + + /** + * Bucket cols. + * + * @param bucketCols the bucket cols + * @return the builder + */ + public Builder bucketCols(List bucketCols, int buckets) { + this.bucketCols = bucketCols; + this.numBuckets = buckets; + return this; + } + + /** + * Storage handler. + * + * @param storageHandler the storage handler + * @return the builder + */ + public Builder storageHandler(String storageHandler) { + this.storageHandler = storageHandler; + return this; + } + + /** + * Location. + * + * @param location the location + * @return the builder + */ + public Builder location(String location) { + this.location = location; + return this; + } + + /** + * Comments. + * + * @param comment the comment + * @return the builder + */ + public Builder comments(String comment) { + this.comment = comment; + return this; + } + + /** + * Checks if is table external. + * + * @param isExternal the is external + * @return the builder + */ + public Builder isTableExternal(boolean isExternal) { + this.isExternal = isExternal; + return this; + } + + /** + * Sort cols. + * + * @param sortCols the sort cols + * @return the builder + */ + public Builder sortCols(ArrayList sortCols) { + this.sortCols = sortCols; + return this; + } + + /** + * Tbl props. + * + * @param tblProps the tbl props + * @return the builder + */ + public Builder tblProps(Map tblProps) { + this.tblProps = tblProps; + return this; + } + + /** + * File format. + * + * @param format the format + * @return the builder + */ + public Builder fileFormat(String format) { + this.fileFormat = format; + return this; + } + + /** + * Builds the HCatCreateTableDesc. + * + * @return HCatCreateTableDesc + * @throws HCatException + */ + public HCatCreateTableDesc build() throws HCatException { + if (this.dbName == null) { + LOG.info("Database name found null. Setting db to :" + + MetaStoreUtils.DEFAULT_DATABASE_NAME); + this.dbName = MetaStoreUtils.DEFAULT_DATABASE_NAME; + } + HCatCreateTableDesc desc = new HCatCreateTableDesc(this.dbName, + this.tableName, this.cols); + desc.ifNotExists = this.ifNotExists; + desc.isExternal = this.isExternal; + desc.comment = this.comment; + desc.partCols = this.partCols; + desc.bucketCols = this.bucketCols; + desc.numBuckets = this.numBuckets; + desc.location = this.location; + desc.tblProps = this.tblProps; + desc.sortCols = this.sortCols; + desc.serde = null; + if (!StringUtils.isEmpty(fileFormat)) { + desc.fileFormat = fileFormat; + if ("SequenceFile".equalsIgnoreCase(fileFormat)) { + desc.inputformat = SequenceFileInputFormat.class.getName(); + desc.outputformat = SequenceFileOutputFormat.class + .getName(); + } else if ("RCFile".equalsIgnoreCase(fileFormat)) { + desc.inputformat = RCFileInputFormat.class.getName(); + desc.outputformat = RCFileOutputFormat.class.getName(); + desc.serde = ColumnarSerDe.class.getName(); + } + desc.storageHandler = StringUtils.EMPTY; + } else if (!StringUtils.isEmpty(storageHandler)) { + desc.storageHandler = storageHandler; + } else { + desc.fileFormat = "TextFile"; + LOG.info("Using text file format for the table."); + desc.inputformat = TextInputFormat.class.getName(); + LOG.info("Table input format:" + desc.inputformat); + desc.outputformat = IgnoreKeyTextOutputFormat.class + .getName(); + LOG.info("Table output format:" + desc.outputformat); + } + return desc; + } + } +} diff --git hcatalog/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatDatabase.java hcatalog/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatDatabase.java new file mode 100644 index 0000000..36125dd --- /dev/null +++ hcatalog/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatDatabase.java @@ -0,0 +1,89 @@ +/** + * 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.hcatalog.api; + +import java.util.Map; + +import org.apache.hadoop.hive.metastore.api.Database; + + +/** + * HCatDatabase is wrapper class around org.apache.hadoop.hive.metastore.api.Database. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.api.HCatDatabase} instead + */ +public class HCatDatabase { + + private String dbName; + private String dbLocation; + private String comment; + private Map props; + + HCatDatabase(Database db) { + this.dbName = db.getName(); + this.props = db.getParameters(); + this.dbLocation = db.getLocationUri(); + this.comment = db.getDescription(); + } + + /** + * Gets the database name. + * + * @return the database name + */ + public String getName() { + return dbName; + } + + /** + * Gets the dB location. + * + * @return the dB location + */ + public String getLocation() { + return dbLocation; + } + + /** + * Gets the comment. + * + * @return the comment + */ + public String getComment() { + return comment; + } + + /** + * Gets the dB properties. + * + * @return the dB properties + */ + public Map getProperties() { + return props; + } + + @Override + public String toString() { + return "HCatDatabase [" + + (dbName != null ? "dbName=" + dbName + ", " : "dbName=null") + + (dbLocation != null ? "dbLocation=" + dbLocation + ", " : "dbLocation=null") + + (comment != null ? "comment=" + comment + ", " : "comment=null") + + (props != null ? "props=" + props : "props=null") + "]"; + } + +} diff --git hcatalog/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatPartition.java hcatalog/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatPartition.java new file mode 100644 index 0000000..34905cb --- /dev/null +++ hcatalog/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatPartition.java @@ -0,0 +1,205 @@ +/** + * 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.hcatalog.api; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.Order; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.data.schema.HCatFieldSchema; +import org.apache.hcatalog.data.schema.HCatSchemaUtils; + +/** + * The HCatPartition is a wrapper around org.apache.hadoop.hive.metastore.api.Partition. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.api.HCatPartition} instead + */ +public class HCatPartition { + + private String tableName; + private String dbName; + private List values; + private List tableCols; + private int createTime; + private int lastAccessTime; + private StorageDescriptor sd; + private Map parameters; + + HCatPartition(Partition partition) throws HCatException { + this.tableName = partition.getTableName(); + this.dbName = partition.getDbName(); + this.createTime = partition.getCreateTime(); + this.lastAccessTime = partition.getLastAccessTime(); + this.parameters = partition.getParameters(); + this.values = partition.getValues(); + this.sd = partition.getSd(); + this.tableCols = new ArrayList(); + for (FieldSchema fs : this.sd.getCols()) { + this.tableCols.add(HCatSchemaUtils.getHCatFieldSchema(fs)); + } + } + + /** + * Gets the table name. + * + * @return the table name + */ + public String getTableName() { + return this.tableName; + } + + /** + * Gets the database name. + * + * @return the database name + */ + public String getDatabaseName() { + return this.dbName; + } + + /** + * Gets the columns of the table. + * + * @return the columns + */ + public List getColumns() { + return this.tableCols; + } + + /** + * Gets the input format. + * + * @return the input format + */ + public String getInputFormat() { + return this.sd.getInputFormat(); + } + + /** + * Gets the output format. + * + * @return the output format + */ + public String getOutputFormat() { + return this.sd.getOutputFormat(); + } + + /** + * Gets the storage handler. + * + * @return the storage handler + */ + public String getStorageHandler() { + return this.sd + .getParameters() + .get(org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_STORAGE); + } + + /** + * Gets the location. + * + * @return the location + */ + public String getLocation() { + return this.sd.getLocation(); + } + + /** + * Gets the serde. + * + * @return the serde + */ + public String getSerDe() { + return this.sd.getSerdeInfo().getSerializationLib(); + } + + public Map getParameters() { + return this.parameters; + } + + /** + * Gets the last access time. + * + * @return the last access time + */ + public int getLastAccessTime() { + return this.lastAccessTime; + } + + /** + * Gets the creates the time. + * + * @return the creates the time + */ + public int getCreateTime() { + return this.createTime; + } + + /** + * Gets the values. + * + * @return the values + */ + public List getValues() { + return this.values; + } + + /** + * Gets the bucket columns. + * + * @return the bucket columns + */ + public List getBucketCols() { + return this.sd.getBucketCols(); + } + + /** + * Gets the number of buckets. + * + * @return the number of buckets + */ + public int getNumBuckets() { + return this.sd.getNumBuckets(); + } + + /** + * Gets the sort columns. + * + * @return the sort columns + */ + public List getSortCols() { + return this.sd.getSortCols(); + } + + @Override + public String toString() { + return "HCatPartition [" + + (tableName != null ? "tableName=" + tableName + ", " : "tableName=null") + + (dbName != null ? "dbName=" + dbName + ", " : "dbName=null") + + (values != null ? "values=" + values + ", " : "values=null") + + "createTime=" + createTime + ", lastAccessTime=" + + lastAccessTime + ", " + (sd != null ? "sd=" + sd + ", " : "sd=null") + + (parameters != null ? "parameters=" + parameters : "parameters=null") + "]"; + } + +} diff --git hcatalog/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatTable.java hcatalog/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatTable.java new file mode 100644 index 0000000..17840cb --- /dev/null +++ hcatalog/webhcat/java-client/src/main/java/org/apache/hcatalog/api/HCatTable.java @@ -0,0 +1,228 @@ +/** + * 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.hcatalog.api; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.Order; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.data.schema.HCatFieldSchema; +import org.apache.hcatalog.data.schema.HCatSchemaUtils; + +/** + * The HCatTable is a wrapper around org.apache.hadoop.hive.metastore.api.Table. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.api.HCatTable} instead + */ +public class HCatTable { + + private String tableName; + private String tabletype; + private List cols; + private List partCols; + private List bucketCols; + private List sortCols; + private int numBuckets; + private String inputFileFormat; + private String outputFileFormat; + private String storageHandler; + private Map tblProps; + private String dbName; + private String serde; + private String location; + + HCatTable(Table hiveTable) throws HCatException { + this.tableName = hiveTable.getTableName(); + this.dbName = hiveTable.getDbName(); + this.tabletype = hiveTable.getTableType(); + cols = new ArrayList(); + for (FieldSchema colFS : hiveTable.getSd().getCols()) { + cols.add(HCatSchemaUtils.getHCatFieldSchema(colFS)); + } + partCols = new ArrayList(); + for (FieldSchema colFS : hiveTable.getPartitionKeys()) { + partCols.add(HCatSchemaUtils.getHCatFieldSchema(colFS)); + } + bucketCols = hiveTable.getSd().getBucketCols(); + sortCols = hiveTable.getSd().getSortCols(); + numBuckets = hiveTable.getSd().getNumBuckets(); + inputFileFormat = hiveTable.getSd().getInputFormat(); + outputFileFormat = hiveTable.getSd().getOutputFormat(); + storageHandler = hiveTable + .getSd() + .getParameters() + .get(org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_STORAGE); + tblProps = hiveTable.getParameters(); + serde = hiveTable.getSd().getSerdeInfo().getSerializationLib(); + location = hiveTable.getSd().getLocation(); + } + + /** + * Gets the table name. + * + * @return the table name + */ + public String getTableName() { + return tableName; + } + + /** + * Gets the db name. + * + * @return the db name + */ + public String getDbName() { + return dbName; + } + + /** + * Gets the columns. + * + * @return the columns + */ + public List getCols() { + return cols; + } + + /** + * Gets the part columns. + * + * @return the part columns + */ + public List getPartCols() { + return partCols; + } + + /** + * Gets the bucket columns. + * + * @return the bucket columns + */ + public List getBucketCols() { + return bucketCols; + } + + /** + * Gets the sort columns. + * + * @return the sort columns + */ + public List getSortCols() { + return sortCols; + } + + /** + * Gets the number of buckets. + * + * @return the number of buckets + */ + public int getNumBuckets() { + return numBuckets; + } + + /** + * Gets the storage handler. + * + * @return the storage handler + */ + public String getStorageHandler() { + return storageHandler; + } + + /** + * Gets the table props. + * + * @return the table props + */ + public Map getTblProps() { + return tblProps; + } + + /** + * Gets the tabletype. + * + * @return the tabletype + */ + public String getTabletype() { + return tabletype; + } + + /** + * Gets the input file format. + * + * @return the input file format + */ + public String getInputFileFormat() { + return inputFileFormat; + } + + /** + * Gets the output file format. + * + * @return the output file format + */ + public String getOutputFileFormat() { + return outputFileFormat; + } + + /** + * Gets the serde lib. + * + * @return the serde lib + */ + public String getSerdeLib() { + return serde; + } + + /** + * Gets the location. + * + * @return the location + */ + public String getLocation() { + return location; + } + + @Override + public String toString() { + return "HCatTable [" + + (tableName != null ? "tableName=" + tableName + ", " : "tableName=null") + + (dbName != null ? "dbName=" + dbName + ", " : "dbName=null") + + (tabletype != null ? "tabletype=" + tabletype + ", " : "tabletype=null") + + (cols != null ? "cols=" + cols + ", " : "cols=null") + + (partCols != null ? "partCols=" + partCols + ", " : "partCols==null") + + (bucketCols != null ? "bucketCols=" + bucketCols + ", " : "bucketCols=null") + + (sortCols != null ? "sortCols=" + sortCols + ", " : "sortCols=null") + + "numBuckets=" + + numBuckets + + ", " + + (inputFileFormat != null ? "inputFileFormat=" + + inputFileFormat + ", " : "inputFileFormat=null") + + (outputFileFormat != null ? "outputFileFormat=" + + outputFileFormat + ", " : "outputFileFormat=null") + + (storageHandler != null ? "storageHandler=" + storageHandler + + ", " : "storageHandler=null") + + (tblProps != null ? "tblProps=" + tblProps + ", " : "tblProps=null") + + (serde != null ? "serde=" + serde + ", " : "serde=") + + (location != null ? "location=" + location : "location=") + "]"; + } +} diff --git hcatalog/webhcat/java-client/src/main/java/org/apache/hcatalog/api/ObjectNotFoundException.java hcatalog/webhcat/java-client/src/main/java/org/apache/hcatalog/api/ObjectNotFoundException.java new file mode 100644 index 0000000..4d4baba --- /dev/null +++ hcatalog/webhcat/java-client/src/main/java/org/apache/hcatalog/api/ObjectNotFoundException.java @@ -0,0 +1,40 @@ +/** + * 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.hcatalog.api; + +import org.apache.hcatalog.common.HCatException; + +/** + * This exception is thrown when a Database, Table or Partition + * specified in an HCatalog query is not found. + * @deprecated Use/modify {@link org.apache.hive.hcatalog.api.ObjectNotFoundException} instead + */ +public class ObjectNotFoundException extends HCatException { + + private static final long serialVersionUID = 1L; + + /** + * @param message Exception message. + * @param cause The wrapped Throwable that caused this exception. + */ + public ObjectNotFoundException(String message, Throwable cause) { + super(message, cause); + } +} diff --git hcatalog/webhcat/java-client/src/test/java/org/apache/hcatalog/api/TestHCatClient.java hcatalog/webhcat/java-client/src/test/java/org/apache/hcatalog/api/TestHCatClient.java new file mode 100644 index 0000000..5b4ae16 --- /dev/null +++ hcatalog/webhcat/java-client/src/test/java/org/apache/hcatalog/api/TestHCatClient.java @@ -0,0 +1,639 @@ +/** + * 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.hcatalog.api; + +import java.math.BigInteger; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.HiveMetaStore; +import org.apache.hadoop.hive.metastore.api.PartitionEventType; +import org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat; +import org.apache.hadoop.hive.ql.io.RCFileInputFormat; +import org.apache.hadoop.hive.ql.io.RCFileOutputFormat; +import org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe; +import org.apache.hadoop.mapred.TextInputFormat; +import org.apache.hcatalog.cli.SemanticAnalysis.HCatSemanticAnalyzer; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hcatalog.common.HCatConstants; +import org.apache.hcatalog.common.HCatException; +import org.apache.hcatalog.data.schema.HCatFieldSchema; +import org.apache.hcatalog.data.schema.HCatFieldSchema.Type; +import org.apache.hcatalog.NoExitSecurityManager; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertArrayEquals; + +/** + * @deprecated Use/modify {@link org.apache.hive.hcatalog.api.TestHCatClient} instead + */ +public class TestHCatClient { + private static final Logger LOG = LoggerFactory.getLogger(TestHCatClient.class); + private static final String msPort = "20101"; + private static HiveConf hcatConf; + private static SecurityManager securityManager; + + private static class RunMS implements Runnable { + + @Override + public void run() { + try { + HiveMetaStore.main(new String[]{"-v", "-p", msPort}); + } catch (Throwable t) { + LOG.error("Exiting. Got exception from metastore: ", t); + } + } + } + + @AfterClass + public static void tearDown() throws Exception { + LOG.info("Shutting down metastore."); + System.setSecurityManager(securityManager); + } + + @BeforeClass + public static void startMetaStoreServer() throws Exception { + + Thread t = new Thread(new RunMS()); + t.start(); + Thread.sleep(40000); + + securityManager = System.getSecurityManager(); + System.setSecurityManager(new NoExitSecurityManager()); + hcatConf = new HiveConf(TestHCatClient.class); + hcatConf.set("hive.metastore.local", "false"); + hcatConf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:" + + msPort); + hcatConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3); + hcatConf.set(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK.varname, + HCatSemanticAnalyzer.class.getName()); + hcatConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, ""); + hcatConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, ""); + hcatConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, + "false"); + System.setProperty(HiveConf.ConfVars.PREEXECHOOKS.varname, " "); + System.setProperty(HiveConf.ConfVars.POSTEXECHOOKS.varname, " "); + } + + @Test + public void testBasicDDLCommands() throws Exception { + String db = "testdb"; + String tableOne = "testTable1"; + String tableTwo = "testTable2"; + HCatClient client = HCatClient.create(new Configuration(hcatConf)); + client.dropDatabase(db, true, HCatClient.DropDBMode.CASCADE); + + HCatCreateDBDesc dbDesc = HCatCreateDBDesc.create(db).ifNotExists(false) + .build(); + client.createDatabase(dbDesc); + List dbNames = client.listDatabaseNamesByPattern("*"); + assertTrue(dbNames.contains("default")); + assertTrue(dbNames.contains(db)); + + HCatDatabase testDb = client.getDatabase(db); + assertTrue(testDb.getComment() == null); + assertTrue(testDb.getProperties().size() == 0); + String warehouseDir = System + .getProperty(ConfVars.METASTOREWAREHOUSE.varname, "/user/hive/warehouse"); + assertTrue(testDb.getLocation().equals( + "file:" + warehouseDir + "/" + db + ".db")); + ArrayList cols = new ArrayList(); + cols.add(new HCatFieldSchema("id", Type.INT, "id comment")); + cols.add(new HCatFieldSchema("value", Type.STRING, "value comment")); + HCatCreateTableDesc tableDesc = HCatCreateTableDesc + .create(db, tableOne, cols).fileFormat("rcfile").build(); + client.createTable(tableDesc); + HCatTable table1 = client.getTable(db, tableOne); + assertTrue(table1.getInputFileFormat().equalsIgnoreCase( + RCFileInputFormat.class.getName())); + assertTrue(table1.getOutputFileFormat().equalsIgnoreCase( + RCFileOutputFormat.class.getName())); + assertTrue(table1.getSerdeLib().equalsIgnoreCase( + ColumnarSerDe.class.getName())); + assertTrue(table1.getCols().equals(cols)); + // Since "ifexists" was not set to true, trying to create the same table + // again + // will result in an exception. + try { + client.createTable(tableDesc); + } catch (HCatException e) { + assertTrue(e.getMessage().contains( + "AlreadyExistsException while creating table.")); + } + + client.dropTable(db, tableOne, true); + HCatCreateTableDesc tableDesc2 = HCatCreateTableDesc.create(db, + tableTwo, cols).build(); + client.createTable(tableDesc2); + HCatTable table2 = client.getTable(db, tableTwo); + assertTrue(table2.getInputFileFormat().equalsIgnoreCase( + TextInputFormat.class.getName())); + assertTrue(table2.getOutputFileFormat().equalsIgnoreCase( + IgnoreKeyTextOutputFormat.class.getName())); + assertTrue(table2.getLocation().equalsIgnoreCase( + "file:" + warehouseDir + "/" + db + ".db/" + tableTwo)); + client.close(); + } + + @Test + public void testPartitionsHCatClientImpl() throws Exception { + HCatClient client = HCatClient.create(new Configuration(hcatConf)); + String dbName = "ptnDB"; + String tableName = "pageView"; + client.dropDatabase(dbName, true, HCatClient.DropDBMode.CASCADE); + + HCatCreateDBDesc dbDesc = HCatCreateDBDesc.create(dbName) + .ifNotExists(true).build(); + client.createDatabase(dbDesc); + ArrayList cols = new ArrayList(); + cols.add(new HCatFieldSchema("userid", Type.INT, "id columns")); + cols.add(new HCatFieldSchema("viewtime", Type.BIGINT, + "view time columns")); + cols.add(new HCatFieldSchema("pageurl", Type.STRING, "")); + cols.add(new HCatFieldSchema("ip", Type.STRING, + "IP Address of the User")); + + ArrayList ptnCols = new ArrayList(); + ptnCols.add(new HCatFieldSchema("dt", Type.STRING, "date column")); + ptnCols.add(new HCatFieldSchema("country", Type.STRING, + "country column")); + HCatCreateTableDesc tableDesc = HCatCreateTableDesc + .create(dbName, tableName, cols).fileFormat("sequencefile") + .partCols(ptnCols).build(); + client.createTable(tableDesc); + + Map firstPtn = new HashMap(); + firstPtn.put("dt", "04/30/2012"); + firstPtn.put("country", "usa"); + HCatAddPartitionDesc addPtn = HCatAddPartitionDesc.create(dbName, + tableName, null, firstPtn).build(); + client.addPartition(addPtn); + + Map secondPtn = new HashMap(); + secondPtn.put("dt", "04/12/2012"); + secondPtn.put("country", "brazil"); + HCatAddPartitionDesc addPtn2 = HCatAddPartitionDesc.create(dbName, + tableName, null, secondPtn).build(); + client.addPartition(addPtn2); + + Map thirdPtn = new HashMap(); + thirdPtn.put("dt", "04/13/2012"); + thirdPtn.put("country", "argentina"); + HCatAddPartitionDesc addPtn3 = HCatAddPartitionDesc.create(dbName, + tableName, null, thirdPtn).build(); + client.addPartition(addPtn3); + + List ptnList = client.listPartitionsByFilter(dbName, + tableName, null); + assertTrue(ptnList.size() == 3); + + HCatPartition ptn = client.getPartition(dbName, tableName, firstPtn); + assertTrue(ptn != null); + + client.dropPartitions(dbName, tableName, firstPtn, true); + ptnList = client.listPartitionsByFilter(dbName, + tableName, null); + assertTrue(ptnList.size() == 2); + + List ptnListTwo = client.listPartitionsByFilter(dbName, + tableName, "country = \"argentina\""); + assertTrue(ptnListTwo.size() == 1); + + client.markPartitionForEvent(dbName, tableName, thirdPtn, + PartitionEventType.LOAD_DONE); + boolean isMarked = client.isPartitionMarkedForEvent(dbName, tableName, + thirdPtn, PartitionEventType.LOAD_DONE); + assertTrue(isMarked); + client.close(); + } + + @Test + public void testDatabaseLocation() throws Exception { + HCatClient client = HCatClient.create(new Configuration(hcatConf)); + String dbName = "locationDB"; + client.dropDatabase(dbName, true, HCatClient.DropDBMode.CASCADE); + + HCatCreateDBDesc dbDesc = HCatCreateDBDesc.create(dbName) + .ifNotExists(true).location("/tmp/" + dbName).build(); + client.createDatabase(dbDesc); + HCatDatabase newDB = client.getDatabase(dbName); + assertTrue(newDB.getLocation().equalsIgnoreCase("file:/tmp/" + dbName)); + client.close(); + } + + @Test + public void testCreateTableLike() throws Exception { + HCatClient client = HCatClient.create(new Configuration(hcatConf)); + String tableName = "tableone"; + String cloneTable = "tabletwo"; + client.dropTable(null, tableName, true); + client.dropTable(null, cloneTable, true); + + ArrayList cols = new ArrayList(); + cols.add(new HCatFieldSchema("id", Type.INT, "id columns")); + cols.add(new HCatFieldSchema("value", Type.STRING, "id columns")); + HCatCreateTableDesc tableDesc = HCatCreateTableDesc + .create(null, tableName, cols).fileFormat("rcfile").build(); + client.createTable(tableDesc); + // create a new table similar to previous one. + client.createTableLike(null, tableName, cloneTable, true, false, null); + List tables = client.listTableNamesByPattern(null, "table*"); + assertTrue(tables.size() == 2); + client.close(); + } + + @Test + public void testRenameTable() throws Exception { + HCatClient client = HCatClient.create(new Configuration(hcatConf)); + String tableName = "temptable"; + String newName = "mytable"; + client.dropTable(null, tableName, true); + client.dropTable(null, newName, true); + ArrayList cols = new ArrayList(); + cols.add(new HCatFieldSchema("id", Type.INT, "id columns")); + cols.add(new HCatFieldSchema("value", Type.STRING, "id columns")); + HCatCreateTableDesc tableDesc = HCatCreateTableDesc + .create(null, tableName, cols).fileFormat("rcfile").build(); + client.createTable(tableDesc); + client.renameTable(null, tableName, newName); + try { + client.getTable(null, tableName); + } catch (HCatException exp) { + assertTrue("Unexpected exception message: " + exp.getMessage(), + exp.getMessage().contains("NoSuchObjectException while fetching table")); + } + HCatTable newTable = client.getTable(null, newName); + assertTrue(newTable != null); + assertTrue(newTable.getTableName().equals(newName)); + client.close(); + } + + @Test + public void testTransportFailure() throws Exception { + HCatClient client = HCatClient.create(new Configuration(hcatConf)); + boolean isExceptionCaught = false; + // Table creation with a long table name causes ConnectionFailureException + final String tableName = "Temptable" + new BigInteger(200, new Random()).toString(2); + + ArrayList cols = new ArrayList(); + cols.add(new HCatFieldSchema("id", Type.INT, "id columns")); + cols.add(new HCatFieldSchema("value", Type.STRING, "id columns")); + try { + HCatCreateTableDesc tableDesc = HCatCreateTableDesc + .create(null, tableName, cols).fileFormat("rcfile").build(); + client.createTable(tableDesc); + } catch (Exception exp) { + isExceptionCaught = true; + assertEquals("Unexpected exception type.", HCatException.class, exp.getClass()); + // The connection was closed, so create a new one. + client = HCatClient.create(new Configuration(hcatConf)); + String newName = "goodTable"; + client.dropTable(null, newName, true); + HCatCreateTableDesc tableDesc2 = HCatCreateTableDesc + .create(null, newName, cols).fileFormat("rcfile").build(); + client.createTable(tableDesc2); + HCatTable newTable = client.getTable(null, newName); + assertTrue(newTable != null); + assertTrue(newTable.getTableName().equalsIgnoreCase(newName)); + + } finally { + client.close(); + assertTrue("The expected exception was never thrown.", isExceptionCaught); + } + } + + @Test + public void testOtherFailure() throws Exception { + HCatClient client = HCatClient.create(new Configuration(hcatConf)); + String tableName = "Temptable"; + boolean isExceptionCaught = false; + client.dropTable(null, tableName, true); + ArrayList cols = new ArrayList(); + cols.add(new HCatFieldSchema("id", Type.INT, "id columns")); + cols.add(new HCatFieldSchema("value", Type.STRING, "id columns")); + try { + HCatCreateTableDesc tableDesc = HCatCreateTableDesc + .create(null, tableName, cols).fileFormat("rcfile").build(); + client.createTable(tableDesc); + // The DB foo is non-existent. + client.getTable("foo", tableName); + } catch (Exception exp) { + isExceptionCaught = true; + assertTrue(exp instanceof HCatException); + String newName = "goodTable"; + client.dropTable(null, newName, true); + HCatCreateTableDesc tableDesc2 = HCatCreateTableDesc + .create(null, newName, cols).fileFormat("rcfile").build(); + client.createTable(tableDesc2); + HCatTable newTable = client.getTable(null, newName); + assertTrue(newTable != null); + assertTrue(newTable.getTableName().equalsIgnoreCase(newName)); + } finally { + client.close(); + assertTrue("The expected exception was never thrown.", isExceptionCaught); + } + } + + @Test + public void testDropTableException() throws Exception { + HCatClient client = HCatClient.create(new Configuration(hcatConf)); + String tableName = "tableToBeDropped"; + boolean isExceptionCaught = false; + client.dropTable(null, tableName, true); + try { + client.dropTable(null, tableName, false); + } catch (Exception exp) { + isExceptionCaught = true; + assertTrue(exp instanceof HCatException); + LOG.info("Drop Table Exception: " + exp.getCause()); + } finally { + client.close(); + assertTrue("The expected exception was never thrown.", isExceptionCaught); + } + } + + @Test + public void testUpdateTableSchema() throws Exception { + try { + HCatClient client = HCatClient.create(new Configuration(hcatConf)); + final String dbName = "testUpdateTableSchema_DBName"; + final String tableName = "testUpdateTableSchema_TableName"; + + client.dropDatabase(dbName, true, HCatClient.DropDBMode.CASCADE); + + client.createDatabase(HCatCreateDBDesc.create(dbName).build()); + List oldSchema = Arrays.asList(new HCatFieldSchema("foo", Type.INT, ""), + new HCatFieldSchema("bar", Type.STRING, "")); + client.createTable(HCatCreateTableDesc.create(dbName, tableName, oldSchema).build()); + + List newSchema = Arrays.asList(new HCatFieldSchema("completely", Type.DOUBLE, ""), + new HCatFieldSchema("new", Type.FLOAT, ""), + new HCatFieldSchema("fields", Type.STRING, "")); + + client.updateTableSchema(dbName, tableName, newSchema); + + assertArrayEquals(newSchema.toArray(), client.getTable(dbName, tableName).getCols().toArray()); + + client.dropDatabase(dbName, false, HCatClient.DropDBMode.CASCADE); + } + catch (Exception exception) { + LOG.error("Unexpected exception.", exception); + assertTrue("Unexpected exception: " + exception.getMessage(), false); + } + } + + @Test + public void testObjectNotFoundException() throws Exception { + try { + + HCatClient client = HCatClient.create(new Configuration(hcatConf)); + String dbName = "testObjectNotFoundException_DBName"; + String tableName = "testObjectNotFoundException_TableName"; + client.dropDatabase(dbName, true, HCatClient.DropDBMode.CASCADE); + + try { // Test that fetching a non-existent db-name yields ObjectNotFound. + client.getDatabase(dbName); + assertTrue("Expected ObjectNotFoundException.", false); + } catch(Exception exception) { + LOG.info("Got exception: ", exception); + assertTrue("Expected ObjectNotFoundException. Got:" + exception.getClass(), + exception instanceof ObjectNotFoundException); + } + + client.createDatabase(HCatCreateDBDesc.create(dbName).build()); + + try { // Test that fetching a non-existent table-name yields ObjectNotFound. + client.getTable(dbName, tableName); + assertTrue("Expected ObjectNotFoundException.", false); + } catch(Exception exception) { + LOG.info("Got exception: ", exception); + assertTrue("Expected ObjectNotFoundException. Got:" + exception.getClass(), + exception instanceof ObjectNotFoundException); + } + + String partitionColumn = "part"; + + List columns = Arrays.asList(new HCatFieldSchema("col", Type.STRING, "")); + ArrayList partitionColumns = new ArrayList( + Arrays.asList(new HCatFieldSchema(partitionColumn, Type.STRING, ""))); + client.createTable(HCatCreateTableDesc.create(dbName, tableName, columns) + .partCols(partitionColumns) + .build()); + + Map partitionSpec = new HashMap(); + partitionSpec.put(partitionColumn, "foobar"); + try { // Test that fetching a non-existent partition yields ObjectNotFound. + client.getPartition(dbName, tableName, partitionSpec); + assertTrue("Expected ObjectNotFoundException.", false); + } catch(Exception exception) { + LOG.info("Got exception: ", exception); + assertTrue("Expected ObjectNotFoundException. Got:" + exception.getClass(), + exception instanceof ObjectNotFoundException); + } + + client.addPartition(HCatAddPartitionDesc.create(dbName, tableName, "", partitionSpec).build()); + + // Test that listPartitionsByFilter() returns an empty-set, if the filter selects no partitions. + assertEquals("Expected empty set of partitions.", + 0, client.listPartitionsByFilter(dbName, tableName, partitionColumn + " < 'foobar'").size()); + + try { // Test that listPartitionsByFilter() throws HCatException if the partition-key is incorrect. + partitionSpec.put("NonExistentKey", "foobar"); + client.getPartition(dbName, tableName, partitionSpec); + assertTrue("Expected HCatException.", false); + } catch(Exception exception) { + LOG.info("Got exception: ", exception); + assertTrue("Expected HCatException. Got:" + exception.getClass(), + exception instanceof HCatException); + assertFalse("Did not expect ObjectNotFoundException.", exception instanceof ObjectNotFoundException); + } + + } + catch (Throwable t) { + LOG.error("Unexpected exception!", t); + assertTrue("Unexpected exception! " + t.getMessage(), false); + } + } + + @Test + public void testGetMessageBusTopicName() throws Exception { + try { + HCatClient client = HCatClient.create(new Configuration(hcatConf)); + String dbName = "testGetMessageBusTopicName_DBName"; + String tableName = "testGetMessageBusTopicName_TableName"; + client.dropDatabase(dbName, true, HCatClient.DropDBMode.CASCADE); + client.createDatabase(HCatCreateDBDesc.create(dbName).build()); + String messageBusTopicName = "MY.topic.name"; + Map tableProperties = new HashMap(1); + tableProperties.put(HCatConstants.HCAT_MSGBUS_TOPIC_NAME, messageBusTopicName); + client.createTable(HCatCreateTableDesc.create(dbName, tableName, Arrays.asList(new HCatFieldSchema("foo", Type.STRING, ""))).tblProps(tableProperties).build()); + + assertEquals("MessageBus topic-name doesn't match!", messageBusTopicName, client.getMessageBusTopicName(dbName, tableName)); + client.dropDatabase(dbName, true, HCatClient.DropDBMode.CASCADE); + client.close(); + } + catch (Exception exception) { + LOG.error("Unexpected exception.", exception); + assertTrue("Unexpected exception:" + exception.getMessage(), false); + } + } + + @Test + public void testPartitionSchema() throws Exception { + try { + HCatClient client = HCatClient.create(new Configuration(hcatConf)); + final String dbName = "myDb"; + final String tableName = "myTable"; + + client.dropDatabase(dbName, true, HCatClient.DropDBMode.CASCADE); + + client.createDatabase(HCatCreateDBDesc.create(dbName).build()); + List columnSchema = Arrays.asList(new HCatFieldSchema("foo", Type.INT, ""), + new HCatFieldSchema("bar", Type.STRING, "")); + + List partitionSchema = Arrays.asList(new HCatFieldSchema("dt", Type.STRING, ""), + new HCatFieldSchema("grid", Type.STRING, "")); + + client.createTable(HCatCreateTableDesc.create(dbName, tableName, columnSchema).partCols(partitionSchema).build()); + + HCatTable table = client.getTable(dbName, tableName); + List partitionColumns = table.getPartCols(); + + assertArrayEquals("Didn't get expected partition-schema back from the HCatTable.", + partitionSchema.toArray(), partitionColumns.toArray()); + client.dropDatabase(dbName, false, HCatClient.DropDBMode.CASCADE); + } + catch (Exception unexpected) { + LOG.error("Unexpected exception!", unexpected); + assertTrue("Unexpected exception! " + unexpected.getMessage(), false); + } + } + + @Test + public void testGetPartitionsWithPartialSpec() throws Exception { + try { + HCatClient client = HCatClient.create(new Configuration(hcatConf)); + final String dbName = "myDb"; + final String tableName = "myTable"; + + client.dropDatabase(dbName, true, HCatClient.DropDBMode.CASCADE); + + client.createDatabase(HCatCreateDBDesc.create(dbName).build()); + List columnSchema = Arrays.asList(new HCatFieldSchema("foo", Type.INT, ""), + new HCatFieldSchema("bar", Type.STRING, "")); + + List partitionSchema = Arrays.asList(new HCatFieldSchema("dt", Type.STRING, ""), + new HCatFieldSchema("grid", Type.STRING, "")); + + client.createTable(HCatCreateTableDesc.create(dbName, tableName, columnSchema).partCols(new ArrayList(partitionSchema)).build()); + + Map partitionSpec = new HashMap(); + partitionSpec.put("grid", "AB"); + partitionSpec.put("dt", "2011_12_31"); + client.addPartition(HCatAddPartitionDesc.create(dbName, tableName, "", partitionSpec).build()); + partitionSpec.put("grid", "AB"); + partitionSpec.put("dt", "2012_01_01"); + client.addPartition(HCatAddPartitionDesc.create(dbName, tableName, "", partitionSpec).build()); + partitionSpec.put("dt", "2012_01_01"); + partitionSpec.put("grid", "OB"); + client.addPartition(HCatAddPartitionDesc.create(dbName, tableName, "", partitionSpec).build()); + partitionSpec.put("dt", "2012_01_01"); + partitionSpec.put("grid", "XB"); + client.addPartition(HCatAddPartitionDesc.create(dbName, tableName, "", partitionSpec).build()); + + Map partialPartitionSpec = new HashMap(); + partialPartitionSpec.put("dt", "2012_01_01"); + + List partitions = client.getPartitions(dbName, tableName, partialPartitionSpec); + assertEquals("Unexpected number of partitions.", 3, partitions.size()); + assertArrayEquals("Mismatched partition.", new String[]{"2012_01_01", "AB"}, partitions.get(0).getValues().toArray()); + assertArrayEquals("Mismatched partition.", new String[]{"2012_01_01", "OB"}, partitions.get(1).getValues().toArray()); + assertArrayEquals("Mismatched partition.", new String[]{"2012_01_01", "XB"}, partitions.get(2).getValues().toArray()); + + client.dropDatabase(dbName, false, HCatClient.DropDBMode.CASCADE); + } + catch (Exception unexpected) { + LOG.error("Unexpected exception!", unexpected); + assertTrue("Unexpected exception! " + unexpected.getMessage(), false); + } + } + + @Test + public void testDropPartitionsWithPartialSpec() throws Exception { + try { + HCatClient client = HCatClient.create(new Configuration(hcatConf)); + final String dbName = "myDb"; + final String tableName = "myTable"; + + client.dropDatabase(dbName, true, HCatClient.DropDBMode.CASCADE); + + client.createDatabase(HCatCreateDBDesc.create(dbName).build()); + List columnSchema = Arrays.asList(new HCatFieldSchema("foo", Type.INT, ""), + new HCatFieldSchema("bar", Type.STRING, "")); + + List partitionSchema = Arrays.asList(new HCatFieldSchema("dt", Type.STRING, ""), + new HCatFieldSchema("grid", Type.STRING, "")); + + client.createTable(HCatCreateTableDesc.create(dbName, tableName, columnSchema).partCols(new ArrayList(partitionSchema)).build()); + + Map partitionSpec = new HashMap(); + partitionSpec.put("grid", "AB"); + partitionSpec.put("dt", "2011_12_31"); + client.addPartition(HCatAddPartitionDesc.create(dbName, tableName, "", partitionSpec).build()); + partitionSpec.put("grid", "AB"); + partitionSpec.put("dt", "2012_01_01"); + client.addPartition(HCatAddPartitionDesc.create(dbName, tableName, "", partitionSpec).build()); + partitionSpec.put("dt", "2012_01_01"); + partitionSpec.put("grid", "OB"); + client.addPartition(HCatAddPartitionDesc.create(dbName, tableName, "", partitionSpec).build()); + partitionSpec.put("dt", "2012_01_01"); + partitionSpec.put("grid", "XB"); + client.addPartition(HCatAddPartitionDesc.create(dbName, tableName, "", partitionSpec).build()); + + Map partialPartitionSpec = new HashMap(); + partialPartitionSpec.put("dt", "2012_01_01"); + + client.dropPartitions(dbName, tableName, partialPartitionSpec, true); + + List partitions = client.getPartitions(dbName, tableName); + assertEquals("Unexpected number of partitions.", 1, partitions.size()); + assertArrayEquals("Mismatched partition.", new String[]{"2011_12_31", "AB"}, partitions.get(0).getValues().toArray()); + + client.dropDatabase(dbName, false, HCatClient.DropDBMode.CASCADE); + } + catch (Exception unexpected) { + LOG.error("Unexpected exception!", unexpected); + assertTrue("Unexpected exception! " + unexpected.getMessage(), false); + } + } + +}