diff --git beeline/src/java/org/apache/hive/beeline/HiveSchemaHelper.java beeline/src/java/org/apache/hive/beeline/HiveSchemaHelper.java index 711f6a8..a4ecc08 100644 --- beeline/src/java/org/apache/hive/beeline/HiveSchemaHelper.java +++ beeline/src/java/org/apache/hive/beeline/HiveSchemaHelper.java @@ -34,6 +34,7 @@ public class HiveSchemaHelper { public static final String DB_DERBY = "derby"; + public static final String DB_HIVE = "hive"; public static final String DB_MSSQL = "mssql"; public static final String DB_MYSQL = "mysql"; public static final String DB_POSTGRACE = "postgres"; @@ -50,15 +51,16 @@ * @throws org.apache.hadoop.hive.metastore.api.MetaException */ public static Connection getConnectionToMetastore(String userName, - String password, boolean printInfo, HiveConf hiveConf) + String password, String url, String driver, boolean printInfo, + HiveConf hiveConf) throws HiveMetaException { try { - String connectionURL = getValidConfVar( - HiveConf.ConfVars.METASTORECONNECTURLKEY, hiveConf); - String driver = getValidConfVar( - HiveConf.ConfVars.METASTORE_CONNECTION_DRIVER, hiveConf); + url = url == null ? getValidConfVar( + HiveConf.ConfVars.METASTORECONNECTURLKEY, hiveConf) : url; + driver = driver == null ? getValidConfVar( + HiveConf.ConfVars.METASTORE_CONNECTION_DRIVER, hiveConf) : driver; if (printInfo) { - System.out.println("Metastore connection URL:\t " + connectionURL); + System.out.println("Metastore connection URL:\t " + url); System.out.println("Metastore Connection Driver :\t " + driver); System.out.println("Metastore connection User:\t " + userName); } @@ -70,7 +72,7 @@ public static Connection getConnectionToMetastore(String userName, Class.forName(driver); // Connect using the JDBC URL and user/pass from conf - return DriverManager.getConnection(connectionURL, userName, password); + return DriverManager.getConnection(url, userName, password); } catch (IOException e) { throw new HiveMetaException("Failed to get schema version.", e); } catch (SQLException e) { @@ -97,7 +99,8 @@ public static String getValidConfVar(HiveConf.ConfVars confVar, HiveConf hiveCon COMMENT } - static final String DEFAUTL_DELIMITER = ";"; + static final String DEFAULT_DELIMITER = ";"; + static final String DEFAULT_QUOTE = "\""; /** * Find the type of given command @@ -140,6 +143,13 @@ public static String getValidConfVar(HiveConf.ConfVars confVar, HiveConf hiveCon public String getDelimiter(); /** + * Get the SQL indentifier quotation character + * + * @return + */ + public String getQuoteCharacter(); + + /** * Clear any client specific tags * * @return @@ -162,6 +172,17 @@ public static String getValidConfVar(HiveConf.ConfVars confVar, HiveConf hiveCon */ public String buildCommand(String scriptDir, String scriptFile) throws IllegalFormatException, IOException; + + /** + * Flatten the nested upgrade script into a buffer + * + * @param scriptDir upgrade script directory + * @param scriptFile upgrade script file + * @param fixQuotes whether to replace quote characters + * @return string of sql commands + */ + public String buildCommand(String scriptDir, String scriptFile, boolean fixQuotes) + throws IllegalFormatException, IOException; } /*** @@ -203,10 +224,16 @@ public boolean isNonExecCommand(String dbCommand) { @Override public String getDelimiter() { - return DEFAUTL_DELIMITER; + return DEFAULT_DELIMITER; } @Override + public String getQuoteCharacter() { + return DEFAULT_QUOTE; + } + + + @Override public String cleanseCommand(String dbCommand) { // strip off the delimiter if (dbCommand.endsWith(getDelimiter())) { @@ -224,6 +251,12 @@ public boolean needsQuotedIdentifier() { @Override public String buildCommand( String scriptDir, String scriptFile) throws IllegalFormatException, IOException { + return buildCommand(scriptDir, scriptFile, false); + } + + @Override + public String buildCommand( + String scriptDir, String scriptFile, boolean fixQuotes) throws IllegalFormatException, IOException { BufferedReader bfReader = new BufferedReader(new FileReader(scriptDir + File.separatorChar + scriptFile)); String currLine; @@ -231,6 +264,11 @@ public String buildCommand( String currentCommand = null; while ((currLine = bfReader.readLine()) != null) { currLine = currLine.trim(); + + if (fixQuotes && !getQuoteCharacter().equals(DEFAULT_QUOTE)) { + currLine = currLine.replace("\\\"", getQuoteCharacter()); + } + if (currLine.isEmpty()) { continue; // skip empty lines } @@ -319,11 +357,46 @@ public boolean isNestedScript(String dbCommand) { } } + // Derby commandline parser + public static class HiveCommandParser extends AbstractCommandParser { + private static String HIVE_NESTING_TOKEN = "SOURCE"; + private final NestedScriptParser nestedDbCommandParser; + + public HiveCommandParser(String dbOpts, String msUsername, String msPassword, + HiveConf hiveConf, String metaDbType) { + super(dbOpts, msUsername, msPassword, hiveConf); + nestedDbCommandParser = getDbCommandParser(metaDbType); + } + + @Override + public String getQuoteCharacter() { + return nestedDbCommandParser.getQuoteCharacter(); + } + + @Override + public String getScriptName(String dbCommand) throws IllegalArgumentException { + + if (!isNestedScript(dbCommand)) { + throw new IllegalArgumentException("Not a script format " + dbCommand); + } + String[] tokens = dbCommand.split(" "); + if (tokens.length != 2) { + throw new IllegalArgumentException("Couldn't parse line " + dbCommand); + } + return tokens[1].replace(";", ""); + } + + @Override + public boolean isNestedScript(String dbCommand) { + return dbCommand.startsWith(HIVE_NESTING_TOKEN); + } + } + // MySQL parser public static class MySqlCommandParser extends AbstractCommandParser { private static final String MYSQL_NESTING_TOKEN = "SOURCE"; private static final String DELIMITER_TOKEN = "DELIMITER"; - private String delimiter = DEFAUTL_DELIMITER; + private String delimiter = DEFAULT_DELIMITER; public MySqlCommandParser(String dbOpts, String msUsername, String msPassword, HiveConf hiveConf) { @@ -365,6 +438,11 @@ public String getDelimiter() { } @Override + public String getQuoteCharacter() { + return "`"; + } + + @Override public boolean isNonExecCommand(String dbCommand) { return super.isNonExecCommand(dbCommand) || (dbCommand.startsWith("/*") && dbCommand.endsWith("*/")) || @@ -474,14 +552,20 @@ public boolean isNestedScript(String dbCommand) { } public static NestedScriptParser getDbCommandParser(String dbName) { - return getDbCommandParser(dbName, null, null, null, null); + return getDbCommandParser(dbName, null); + } + + public static NestedScriptParser getDbCommandParser(String dbName, String metaDbName) { + return getDbCommandParser(dbName, null, null, null, null, metaDbName); } public static NestedScriptParser getDbCommandParser(String dbName, String dbOpts, String msUsername, String msPassword, - HiveConf hiveConf) { + HiveConf hiveConf, String metaDbType) { if (dbName.equalsIgnoreCase(DB_DERBY)) { return new DerbyCommandParser(dbOpts, msUsername, msPassword, hiveConf); + } else if (dbName.equalsIgnoreCase(DB_HIVE)) { + return new HiveCommandParser(dbOpts, msUsername, msPassword, hiveConf, metaDbType); } else if (dbName.equalsIgnoreCase(DB_MSSQL)) { return new MSSQLCommandParser(dbOpts, msUsername, msPassword, hiveConf); } else if (dbName.equalsIgnoreCase(DB_MYSQL)) { diff --git beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java index 7dd4d5f..2a022d3 100644 --- beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java +++ beeline/src/java/org/apache/hive/beeline/HiveSchemaTool.java @@ -71,24 +71,28 @@ private boolean dryRun = false; private boolean verbose = false; private String dbOpts = null; + private String url = null; + private String driver = null; private URI[] validationServers = null; // The list of servers the database/partition/table can locate on private final HiveConf hiveConf; private final String dbType; + private final String metaDbType; private final MetaStoreSchemaInfo metaStoreSchemaInfo; static final private Logger LOG = LoggerFactory.getLogger(HiveSchemaTool.class.getName()); - public HiveSchemaTool(String dbType) throws HiveMetaException { - this(System.getenv("HIVE_HOME"), new HiveConf(HiveSchemaTool.class), dbType); + public HiveSchemaTool(String dbType, String metaDbType) throws HiveMetaException { + this(System.getenv("HIVE_HOME"), new HiveConf(HiveSchemaTool.class), dbType, metaDbType); } - public HiveSchemaTool(String hiveHome, HiveConf hiveConf, String dbType) + public HiveSchemaTool(String hiveHome, HiveConf hiveConf, String dbType, String metaDbType) throws HiveMetaException { if (hiveHome == null || hiveHome.isEmpty()) { throw new HiveMetaException("No Hive home directory provided"); } this.hiveConf = hiveConf; this.dbType = dbType; + this.metaDbType = metaDbType; this.metaStoreSchemaInfo = new MetaStoreSchemaInfo(hiveHome, dbType); } @@ -96,6 +100,14 @@ public HiveConf getHiveConf() { return hiveConf; } + public void setUrl(String url) { + this.url = url; + } + + public void setDriver(String driver) { + this.driver = driver; + } + public void setUserName(String userName) { this.userName = userName; } @@ -135,12 +147,17 @@ private static void printAndExit(Options cmdLineOptions) { Connection getConnectionToMetastore(boolean printInfo) throws HiveMetaException { return HiveSchemaHelper.getConnectionToMetastore(userName, - passWord, printInfo, hiveConf); + passWord, url, driver, printInfo, hiveConf); + } + + private NestedScriptParser getDbCommandParser(String dbType, String metaDbType) { + return HiveSchemaHelper.getDbCommandParser(dbType, dbOpts, userName, + passWord, hiveConf, metaDbType); } private NestedScriptParser getDbCommandParser(String dbType) { return HiveSchemaHelper.getDbCommandParser(dbType, dbOpts, userName, - passWord, hiveConf); + passWord, hiveConf, null); } /*** @@ -936,9 +953,9 @@ private void runPreUpgrade(String scriptDir, String scriptFile) { */ private void runBeeLine(String scriptDir, String scriptFile) throws IOException, HiveMetaException { - NestedScriptParser dbCommandParser = getDbCommandParser(dbType); + NestedScriptParser dbCommandParser = getDbCommandParser(dbType, metaDbType); // expand the nested script - String sqlCommands = dbCommandParser.buildCommand(scriptDir, scriptFile); + String sqlCommands = dbCommandParser.buildCommand(scriptDir, scriptFile, metaDbType != null); File tmpFile = File.createTempFile("schematool", ".sql"); tmpFile.deleteOnExit(); @@ -954,7 +971,8 @@ private void runBeeLine(String scriptDir, String scriptFile) // Generate the beeline args per hive conf and execute the given script public void runBeeLine(String sqlScriptFile) throws IOException { - CommandBuilder builder = new CommandBuilder(hiveConf, userName, passWord, sqlScriptFile); + CommandBuilder builder = new CommandBuilder(hiveConf, url, driver, + userName, passWord, sqlScriptFile); // run the script using Beeline try (BeeLine beeLine = new BeeLine()) { @@ -980,11 +998,16 @@ public void runBeeLine(String sqlScriptFile) throws IOException { private final String userName; private final String password; private final String sqlScriptFile; + private final String driver; + private final String url; - CommandBuilder(HiveConf hiveConf, String userName, String password, String sqlScriptFile) { + CommandBuilder(HiveConf hiveConf, String url, String driver, + String userName, String password, String sqlScriptFile) { this.hiveConf = hiveConf; this.userName = userName; this.password = password; + this.url = url; + this.driver = driver; this.sqlScriptFile = sqlScriptFile; } @@ -998,10 +1021,14 @@ String buildToLog() throws IOException { } private String[] argsWith(String password) throws IOException { - return new String[] { "-u", - HiveSchemaHelper.getValidConfVar(ConfVars.METASTORECONNECTURLKEY, hiveConf), "-d", - HiveSchemaHelper.getValidConfVar(ConfVars.METASTORE_CONNECTION_DRIVER, hiveConf), "-n", - userName, "-p", password, "-f", sqlScriptFile }; + return new String[] + { + "-u", url == null ? HiveSchemaHelper.getValidConfVar(ConfVars.METASTORECONNECTURLKEY, hiveConf) : url, + "-d", driver == null ? HiveSchemaHelper.getValidConfVar(ConfVars.METASTORE_CONNECTION_DRIVER, hiveConf) : driver, + "-n", userName, + "-p", password, + "-f", sqlScriptFile + }; } private void logScript() throws IOException { @@ -1049,6 +1076,15 @@ private static void initOptions(Options cmdLineOptions) { Option dbTypeOpt = OptionBuilder.withArgName("databaseType") .hasArgs().withDescription("Metastore database type") .create("dbType"); + Option metaDbTypeOpt = OptionBuilder.withArgName("metaDatabaseType") + .hasArgs().withDescription("Used only if upgrading the system catalog for hive") + .create("metaDbType"); + Option urlOpt = OptionBuilder.withArgName("url") + .hasArgs().withDescription("connection url to the database") + .create("url"); + Option driverOpt = OptionBuilder.withArgName("driver") + .hasArgs().withDescription("driver name for connection") + .create("driver"); Option dbOpts = OptionBuilder.withArgName("databaseOpts") .hasArgs().withDescription("Backend DB specific options") .create("dbOpts"); @@ -1063,6 +1099,9 @@ private static void initOptions(Options cmdLineOptions) { cmdLineOptions.addOption(passwdOpt); cmdLineOptions.addOption(dbTypeOpt); cmdLineOptions.addOption(verboseOpt); + cmdLineOptions.addOption(metaDbTypeOpt); + cmdLineOptions.addOption(urlOpt); + cmdLineOptions.addOption(driverOpt); cmdLineOptions.addOption(dbOpts); cmdLineOptions.addOption(serversOpt); cmdLineOptions.addOptionGroup(optGroup); @@ -1072,6 +1111,7 @@ public static void main(String[] args) { CommandLineParser parser = new GnuParser(); CommandLine line = null; String dbType = null; + String metaDbType = null; String schemaVer = null; Options cmdLineOptions = new Options(); @@ -1093,6 +1133,7 @@ public static void main(String[] args) { if (line.hasOption("dbType")) { dbType = line.getOptionValue("dbType"); if ((!dbType.equalsIgnoreCase(HiveSchemaHelper.DB_DERBY) && + !dbType.equalsIgnoreCase(HiveSchemaHelper.DB_HIVE) && !dbType.equalsIgnoreCase(HiveSchemaHelper.DB_MSSQL) && !dbType.equalsIgnoreCase(HiveSchemaHelper.DB_MYSQL) && !dbType.equalsIgnoreCase(HiveSchemaHelper.DB_POSTGRACE) && !dbType @@ -1105,9 +1146,25 @@ public static void main(String[] args) { printAndExit(cmdLineOptions); } + if (line.hasOption("metaDbType")) { + metaDbType = line.getOptionValue("metaDbType"); + if ((!metaDbType.equalsIgnoreCase(HiveSchemaHelper.DB_DERBY) && + !metaDbType.equalsIgnoreCase(HiveSchemaHelper.DB_MSSQL) && + !metaDbType.equalsIgnoreCase(HiveSchemaHelper.DB_MYSQL) && + !metaDbType.equalsIgnoreCase(HiveSchemaHelper.DB_POSTGRACE) && !metaDbType + .equalsIgnoreCase(HiveSchemaHelper.DB_ORACLE))) { + System.err.println("Unsupported metaDbType " + metaDbType); + printAndExit(cmdLineOptions); + } + } else if (dbType.equalsIgnoreCase(HiveSchemaHelper.DB_HIVE)) { + System.err.println("no metaDbType supplied"); + printAndExit(cmdLineOptions); + } + + System.setProperty(HiveConf.ConfVars.METASTORE_SCHEMA_VERIFICATION.varname, "true"); try { - HiveSchemaTool schemaTool = new HiveSchemaTool(dbType); + HiveSchemaTool schemaTool = new HiveSchemaTool(dbType, metaDbType); if (line.hasOption("userName")) { schemaTool.setUserName(line.getOptionValue("userName")); @@ -1125,6 +1182,12 @@ public static void main(String[] args) { throw new HiveMetaException("Error getting metastore password", err); } } + if (line.hasOption("url")) { + schemaTool.setUrl(line.getOptionValue("url")); + } + if (line.hasOption("driver")) { + schemaTool.setDriver(line.getOptionValue("driver")); + } if (line.hasOption("dryRun")) { schemaTool.setDryRun(true); } diff --git beeline/src/test/org/apache/hive/beeline/TestHiveSchemaTool.java beeline/src/test/org/apache/hive/beeline/TestHiveSchemaTool.java index 4cd5124..716bce7 100644 --- beeline/src/test/org/apache/hive/beeline/TestHiveSchemaTool.java +++ beeline/src/test/org/apache/hive/beeline/TestHiveSchemaTool.java @@ -64,7 +64,7 @@ public void setup() throws IOException { if (!file.exists()) { file.createNewFile(); } - builder = new HiveSchemaTool.CommandBuilder(hiveConf, "testUser", pasword, scriptFile); + builder = new HiveSchemaTool.CommandBuilder(hiveConf, null, null, "testUser", pasword, scriptFile); } @After @@ -87,4 +87,4 @@ public void shouldReturnActualPassword() throws IOException { String[] strings = builder.buildToRun(); assertTrue(Arrays.asList(strings).contains(pasword)); } -} \ No newline at end of file +} diff --git common/src/java/org/apache/hadoop/hive/conf/HiveConf.java common/src/java/org/apache/hadoop/hive/conf/HiveConf.java index 99c26ce..2d2f8ee 100644 --- common/src/java/org/apache/hadoop/hive/conf/HiveConf.java +++ common/src/java/org/apache/hadoop/hive/conf/HiveConf.java @@ -597,6 +597,8 @@ private static void populateLlapDaemonVarsSet(Set llapDaemonVarsSetLocal HADOOPNUMREDUCERS("mapreduce.job.reduces", -1, "", true), // Metastore stuff. Be sure to update HiveConf.metaVars when you add something here! + METASTOREDBTYPE("hive.metastore.db.type", "DERBY", new StringSet("DERBY", "ORACLE", "MYSQL", "MSSQL", "POSTGRES"), + "Type of database used for the metastore"), METASTOREWAREHOUSE("hive.metastore.warehouse.dir", "/user/hive/warehouse", "location of default database for the warehouse"), METASTOREURIS("hive.metastore.uris", "", diff --git druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java index daee2fe..f76fd8d 100644 --- druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java +++ druid-handler/src/java/org/apache/hadoop/hive/druid/DruidStorageHandler.java @@ -191,6 +191,12 @@ public void configureInputJobProperties(TableDesc tableDesc, Map } @Override + public void configureInputJobCredentials(TableDesc tableDesc, Map jobSecrets + ) { + + } + + @Override public void preCreateTable(Table table) throws MetaException { // Do safety checks if (MetaStoreUtils.isExternalTable(table) && !StringUtils diff --git itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java index 604c234..6053848 100644 --- itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java +++ itests/hive-unit/src/test/java/org/apache/hive/beeline/TestSchemaTool.java @@ -57,7 +57,7 @@ protected void setUp() throws Exception { "jdbc:derby:" + testMetastoreDB + ";create=true"); hiveConf = new HiveConf(this.getClass()); schemaTool = new HiveSchemaTool( - System.getProperty("test.tmp.dir", "target/tmp"), hiveConf, "derby"); + System.getProperty("test.tmp.dir", "target/tmp"), hiveConf, "derby", null); schemaTool.setUserName( schemaTool.getHiveConf().get(HiveConf.ConfVars.METASTORE_CONNECTION_USER_NAME.varname)); schemaTool.setPassWord(ShimLoader.getHadoopShims().getPassword(schemaTool.getHiveConf(), @@ -590,7 +590,7 @@ public void testPostgresFilter() throws Exception { NestedScriptParser dbOptParser = HiveSchemaHelper.getDbCommandParser( "postgres", PostgresCommandParser.POSTGRES_SKIP_STANDARD_STRINGS_DBOPT, - null, null, null); + null, null, null, null); expectedSQL = StringUtils.join( expectedScriptWithOptionAbsent, System.getProperty("line.separator")) + System.getProperty("line.separator"); diff --git itests/src/test/resources/testconfiguration.properties itests/src/test/resources/testconfiguration.properties index 5ab3076..f0aebda 100644 --- itests/src/test/resources/testconfiguration.properties +++ itests/src/test/resources/testconfiguration.properties @@ -583,6 +583,7 @@ minillaplocal.query.files=acid_globallimit.q,\ subquery_select.q, \ subquery_shared_alias.q, \ subquery_null_agg.q,\ + sysdb.q,\ table_access_keys_stats.q,\ tez_bmj_schema_evolution.q,\ tez_dml.q,\ diff --git jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/JdbcInputFormat.java jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/JdbcInputFormat.java index bfa7a26..6def148 100644 --- jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/JdbcInputFormat.java +++ jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/JdbcInputFormat.java @@ -59,15 +59,23 @@ @Override public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException { try { - if (numSplits <= 0) { - numSplits = 1; - } + LOGGER.debug("Creating {} input splits", numSplits); + if (dbAccessor == null) { dbAccessor = DatabaseAccessorFactory.getAccessor(job); } int numRecords = dbAccessor.getTotalNumberOfRecords(job); + + if (numRecords < numSplits) { + numSplits = numRecords; + } + + if (numSplits <= 0) { + numSplits = 1; + } + int numRecordsPerSplit = numRecords / numSplits; int numSplitsWithExtraRecords = numRecords % numSplits; @@ -86,6 +94,7 @@ offset += numRecordsInThisSplit; } + dbAccessor = null; return splits; } catch (Exception e) { diff --git jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/JdbcRecordReader.java jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/JdbcRecordReader.java index 0a24bd9..8321a66 100644 --- jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/JdbcRecordReader.java +++ jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/JdbcRecordReader.java @@ -15,6 +15,7 @@ package org.apache.hive.storage.jdbc; import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.MapWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapred.JobConf; @@ -63,7 +64,8 @@ public boolean next(LongWritable key, MapWritable value) throws IOException { Map record = iterator.next(); if ((record != null) && (!record.isEmpty())) { for (Entry entry : record.entrySet()) { - value.put(new Text(entry.getKey()), new Text(entry.getValue())); + value.put(new Text(entry.getKey()), + entry.getValue() == null ? NullWritable.get() : new Text(entry.getValue())); } return true; } diff --git jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/JdbcSerDe.java jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/JdbcSerDe.java index f35c33d..e785e9c 100644 --- jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/JdbcSerDe.java +++ jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/JdbcSerDe.java @@ -23,6 +23,7 @@ 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.io.NullWritable; import org.apache.hadoop.io.MapWritable; import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Writable; @@ -61,7 +62,6 @@ public void initialize(Configuration conf, Properties tbl) throws SerDeException try { LOGGER.debug("Initializing the SerDe"); - // Hive cdh-4.3 does not provide the properties object on all calls if (tbl.containsKey(JdbcStorageConfig.DATABASE_TYPE.getPropertyName())) { Configuration tableConfig = JdbcStorageConfigManager.convertPropertiesToConfiguration(tbl); @@ -126,7 +126,7 @@ public Object deserialize(Writable blob) throws SerDeException { for (int i = 0; i < numColumns; i++) { columnKey.set(columnNames.get(i)); Writable value = input.get(columnKey); - if (value == null) { + if (value == null || value instanceof NullWritable) { row.add(null); } else { diff --git jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/JdbcStorageHandler.java jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/JdbcStorageHandler.java index 946ee0c..daa219b 100644 --- jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/JdbcStorageHandler.java +++ jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/JdbcStorageHandler.java @@ -24,6 +24,8 @@ import org.apache.hadoop.mapred.InputFormat; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.OutputFormat; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.hive.storage.jdbc.conf.JdbcStorageConfigManager; @@ -32,6 +34,7 @@ public class JdbcStorageHandler implements HiveStorageHandler { + private static final Logger LOGGER = LoggerFactory.getLogger(JdbcStorageHandler.class); private Configuration conf; @@ -72,27 +75,32 @@ public HiveMetaHook getMetaHook() { return null; } - @Override - public void configureTableJobProperties(TableDesc tableDesc, Map jobProperties) { + public void configureInputJobProperties(TableDesc tableDesc, Map jobProperties) { + LOGGER.debug("Adding properties to input job conf"); Properties properties = tableDesc.getProperties(); JdbcStorageConfigManager.copyConfigurationToJob(properties, jobProperties); } + @Override + public void configureInputJobCredentials(TableDesc tableDesc, Map jobSecrets) { + LOGGER.debug("Adding secrets to input job conf"); + Properties properties = tableDesc.getProperties(); + JdbcStorageConfigManager.copySecretsToJob(properties, jobSecrets); + } @Override - public void configureInputJobProperties(TableDesc tableDesc, Map jobProperties) { + public void configureTableJobProperties(TableDesc tableDesc, Map jobProperties) { + LOGGER.debug("Adding properties to input job conf"); Properties properties = tableDesc.getProperties(); JdbcStorageConfigManager.copyConfigurationToJob(properties, jobProperties); } - @Override public void configureOutputJobProperties(TableDesc tableDesc, Map jobProperties) { // Nothing to do here... } - @Override public HiveAuthorizationProvider getAuthorizationProvider() throws HiveException { return null; diff --git jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/conf/DatabaseType.java jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/conf/DatabaseType.java index a2bdbe4..c4e97ba 100644 --- jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/conf/DatabaseType.java +++ jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/conf/DatabaseType.java @@ -17,5 +17,9 @@ public enum DatabaseType { MYSQL, H2, - DERBY + DERBY, + ORACLE, + POSTGRES, + MSSQL, + METASTORE } diff --git jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/conf/JdbcStorageConfigManager.java jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/conf/JdbcStorageConfigManager.java index 5267cda..14612ed 100644 --- jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/conf/JdbcStorageConfigManager.java +++ jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/conf/JdbcStorageConfigManager.java @@ -14,10 +14,19 @@ */ package org.apache.hive.storage.jdbc.conf; +import java.io.IOException; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hive.storage.jdbc.conf.DatabaseType; + import org.apache.hadoop.conf.Configuration; import org.apache.hive.storage.jdbc.QueryConditionBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.util.EnumSet; import java.util.Map; import java.util.Map.Entry; @@ -28,28 +37,38 @@ */ public class JdbcStorageConfigManager { + private static final Logger LOGGER = LoggerFactory.getLogger(JdbcStorageConfigManager.class); public static final String CONFIG_PREFIX = "hive.sql"; + public static final String CONFIG_PWD = CONFIG_PREFIX + ".dbcp.password"; private static final EnumSet DEFAULT_REQUIRED_PROPERTIES = EnumSet.of(JdbcStorageConfig.DATABASE_TYPE, - JdbcStorageConfig.JDBC_URL, - JdbcStorageConfig.JDBC_DRIVER_CLASS, JdbcStorageConfig.QUERY); - private JdbcStorageConfigManager() { } - public static void copyConfigurationToJob(Properties props, Map jobProps) { checkRequiredPropertiesAreDefined(props); + resolveMetadata(props); for (Entry entry : props.entrySet()) { - jobProps.put(String.valueOf(entry.getKey()), String.valueOf(entry.getValue())); + if (!String.valueOf(entry.getKey()).equals(CONFIG_PWD)) { + jobProps.put(String.valueOf(entry.getKey()), String.valueOf(entry.getValue())); + } } } + public static void copySecretsToJob(Properties props, Map jobSecrets) { + checkRequiredPropertiesAreDefined(props); + resolveMetadata(props); + String secret = props.getProperty(CONFIG_PWD); + if (secret != null) { + jobSecrets.put(CONFIG_PWD, secret); + } + } public static Configuration convertPropertiesToConfiguration(Properties props) { checkRequiredPropertiesAreDefined(props); + resolveMetadata(props); Configuration conf = new Configuration(); for (Entry entry : props.entrySet()) { @@ -94,4 +113,57 @@ private static boolean isEmptyString(String value) { return ((value == null) || (value.trim().isEmpty())); } + private static void resolveMetadata(Properties props) { + try { + DatabaseType dbType = DatabaseType.valueOf( + props.getProperty(JdbcStorageConfig.DATABASE_TYPE.getPropertyName())); + + LOGGER.debug("Resolving db type: {}", dbType.toString()); + + if (dbType == DatabaseType.METASTORE) { + HiveConf hconf = Hive.get().getConf(); + props.setProperty(JdbcStorageConfig.JDBC_URL.getPropertyName(), + getMetastoreConnectionURL(hconf)); + props.setProperty(JdbcStorageConfig.JDBC_DRIVER_CLASS.getPropertyName(), + getMetastoreDriver(hconf)); + + String user = getMetastoreJdbcUser(hconf); + if (user != null) { + props.setProperty("hive.sql.dbcp.username", user); + } + + String pwd = getMetastoreJdbcPasswd(hconf); + if (pwd != null) { + props.setProperty(CONFIG_PWD, pwd); + } + props.setProperty(JdbcStorageConfig.DATABASE_TYPE.getPropertyName(), + getMetastoreDatabaseType(hconf)); + } + } catch (Exception e) {} + } + + private static String getMetastoreDatabaseType(HiveConf conf) { + return conf.getVar(HiveConf.ConfVars.METASTOREDBTYPE); + } + + private static String getMetastoreConnectionURL(HiveConf conf) { + return conf.getVar(HiveConf.ConfVars.METASTORECONNECTURLKEY); + } + + private static String getMetastoreDriver(HiveConf conf) { + return conf.getVar(HiveConf.ConfVars.METASTORE_CONNECTION_DRIVER); + } + + private static String getMetastoreJdbcUser(HiveConf conf) { + return conf.getVar(HiveConf.ConfVars.METASTORE_CONNECTION_USER_NAME); + } + + private static String getMetastoreJdbcPasswd(HiveConf conf) { + try { + return ShimLoader.getHadoopShims().getPassword(conf, + HiveConf.ConfVars.METASTOREPWD.varname); + } catch (IOException io) { + } + return null; + } } diff --git jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/dao/GenericJdbcDatabaseAccessor.java jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/dao/GenericJdbcDatabaseAccessor.java index b655aec..178c97d 100644 --- jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/dao/GenericJdbcDatabaseAccessor.java +++ jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/dao/GenericJdbcDatabaseAccessor.java @@ -16,6 +16,10 @@ import org.apache.commons.dbcp.BasicDataSourceFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.security.Credentials; +import org.apache.hadoop.security.UserGroupInformation; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -45,6 +49,7 @@ protected static final int DEFAULT_FETCH_SIZE = 1000; protected static final Logger LOGGER = LoggerFactory.getLogger(GenericJdbcDatabaseAccessor.class); protected DataSource dbcpDataSource = null; + protected static final Text DBCP_PWD = new Text(DBCP_CONFIG_PREFIX + ".password"); public GenericJdbcDatabaseAccessor() { @@ -97,7 +102,7 @@ public int getTotalNumberOfRecords(Configuration conf) throws HiveJdbcDatabaseAc initializeDatabaseConnection(conf); String sql = JdbcStorageConfigManager.getQueryToExecute(conf); String countQuery = "SELECT COUNT(*) FROM (" + sql + ") tmptable"; - LOGGER.debug("Query to execute is [{}]", countQuery); + LOGGER.info("Query to execute is [{}]", countQuery); conn = dbcpDataSource.getConnection(); ps = conn.prepareStatement(countQuery); @@ -135,7 +140,7 @@ public int getTotalNumberOfRecords(Configuration conf) throws HiveJdbcDatabaseAc initializeDatabaseConnection(conf); String sql = JdbcStorageConfigManager.getQueryToExecute(conf); String limitQuery = addLimitAndOffsetToQuery(sql, limit, offset); - LOGGER.debug("Query to execute is [{}]", limitQuery); + LOGGER.info("Query to execute is [{}]", limitQuery); conn = dbcpDataSource.getConnection(); ps = conn.prepareStatement(limitQuery, ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY); @@ -216,7 +221,7 @@ protected void initializeDatabaseConnection(Configuration conf) throws Exception } - protected Properties getConnectionPoolProperties(Configuration conf) { + protected Properties getConnectionPoolProperties(Configuration conf) throws Exception { // Create the default properties object Properties dbProperties = getDefaultDBCPProperties(); @@ -228,6 +233,13 @@ protected Properties getConnectionPoolProperties(Configuration conf) { } } + // handle password + Credentials credentials = UserGroupInformation.getCurrentUser().getCredentials(); + if (credentials.getSecretKey(DBCP_PWD) != null) { + LOGGER.info("found token in credentials"); + dbProperties.put(DBCP_PWD,new String(credentials.getSecretKey(DBCP_PWD))); + } + // essential properties that shouldn't be overridden by users dbProperties.put("url", conf.get(JdbcStorageConfig.JDBC_URL.getPropertyName())); dbProperties.put("driverClassName", conf.get(JdbcStorageConfig.JDBC_DRIVER_CLASS.getPropertyName())); diff --git jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/dao/JdbcRecordIterator.java jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/dao/JdbcRecordIterator.java index 4262502..8938766 100644 --- jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/dao/JdbcRecordIterator.java +++ jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/dao/JdbcRecordIterator.java @@ -66,9 +66,6 @@ public boolean hasNext() { for (int i = 0; i < numColumns; i++) { String key = metadata.getColumnName(i + 1); String value = rs.getString(i + 1); - if (value == null) { - value = NullWritable.get().toString(); - } record.put(key, value); } diff --git jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/dao/MySqlDatabaseAccessor.java jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/dao/MySqlDatabaseAccessor.java index 7d821d8..86fde7c 100644 --- jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/dao/MySqlDatabaseAccessor.java +++ jdbc-handler/src/main/java/org/apache/hive/storage/jdbc/dao/MySqlDatabaseAccessor.java @@ -26,7 +26,7 @@ protected String addLimitAndOffsetToQuery(String sql, int limit, int offset) { return addLimitToQuery(sql, limit); } else { - return sql + " LIMIT " + limit + "," + offset; + return sql + " LIMIT " + offset + "," + limit; } } diff --git jdbc-handler/src/test/java/org/apache/hive/config/JdbcStorageConfigManagerTest.java jdbc-handler/src/test/java/org/apache/hive/config/JdbcStorageConfigManagerTest.java index c950831..99295a3 100644 --- jdbc-handler/src/test/java/org/apache/hive/config/JdbcStorageConfigManagerTest.java +++ jdbc-handler/src/test/java/org/apache/hive/config/JdbcStorageConfigManagerTest.java @@ -20,6 +20,7 @@ import static org.junit.Assert.assertThat; import org.junit.Test; +import org.junit.Ignore; import org.apache.hive.storage.jdbc.conf.DatabaseType; import org.apache.hive.storage.jdbc.conf.JdbcStorageConfig; @@ -51,7 +52,8 @@ public void testWithAllRequiredSettingsDefined() { } - @Test(expected = IllegalArgumentException.class) + // since metastore connections don't require the url, this is allowable. + @Ignore @Test(expected = IllegalArgumentException.class) public void testWithJdbcUrlMissing() { Properties props = new Properties(); props.put(JdbcStorageConfig.DATABASE_TYPE.getPropertyName(), DatabaseType.MYSQL.toString()); diff --git llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java index a662c75..2c88abc 100644 --- llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java +++ llap-server/src/java/org/apache/hadoop/hive/llap/cli/LlapServiceDriver.java @@ -89,7 +89,9 @@ protected static final Logger LOG = LoggerFactory.getLogger(LlapServiceDriver.class.getName()); private static final String[] DEFAULT_AUX_CLASSES = new String[] { - "org.apache.hive.hcatalog.data.JsonSerDe","org.apache.hadoop.hive.druid.DruidStorageHandler" }; + "org.apache.hive.hcatalog.data.JsonSerDe","org.apache.hadoop.hive.druid.DruidStorageHandler", + "org.apache.hive.storage.jdbc.JdbcStorageHandler" + }; private static final String HBASE_SERDE_CLASS = "org.apache.hadoop.hive.hbase.HBaseSerDe"; private static final String[] NEEDED_CONFIGS = LlapDaemonConfiguration.DAEMON_CONFIGS; private static final String[] OPTIONAL_CONFIGS = LlapDaemonConfiguration.SSL_DAEMON_CONFIGS; diff --git metastore/scripts/upgrade/hive/hive-schema-3.0.0.hive.sql metastore/scripts/upgrade/hive/hive-schema-3.0.0.hive.sql new file mode 100644 index 0000000..1db0f4c --- /dev/null +++ metastore/scripts/upgrade/hive/hive-schema-3.0.0.hive.sql @@ -0,0 +1,1334 @@ +-- HIVE system db + +CREATE DATABASE SYS; + +USE SYS; + +CREATE TABLE IF NOT EXISTS `BUCKETING_COLS` ( + `SD_ID` bigint, + `BUCKET_COL_NAME` string, + `INTEGER_IDX` int, + CONSTRAINT `SYS_PK_BUCKETING_COLS` PRIMARY KEY (`SD_ID`,`INTEGER_IDX`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"SD_ID\", + \"BUCKET_COL_NAME\", + \"INTEGER_IDX\" +FROM + \"BUCKETING_COLS\"" +); + +CREATE TABLE IF NOT EXISTS `CDS` ( + `CD_ID` bigint, + CONSTRAINT `SYS_PK_CDS` PRIMARY KEY (`CD_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"CD_ID\" +FROM + \"CDS\"" +); + +CREATE TABLE IF NOT EXISTS `COLUMNS_V2` ( + `CD_ID` bigint, + `COMMENT` string, + `COLUMN_NAME` string, + `TYPE_NAME` string, + `INTEGER_IDX` int, + CONSTRAINT `SYS_PK_COLUMN_V2` PRIMARY KEY (`CD_ID`,`COLUMN_NAME`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"CD_ID\", + \"COMMENT\", + \"COLUMN_NAME\", + \"TYPE_NAME\", + \"INTEGER_IDX\" +FROM + \"COLUMNS_V2\"" +); + +CREATE TABLE IF NOT EXISTS `DATABASE_PARAMS` ( + `DB_ID` bigint, + `PARAM_KEY` string, + `PARAM_VALUE` string, + CONSTRAINT `SYS_PK_DATABASE_PARAMS` PRIMARY KEY (`DB_ID`,`PARAM_KEY`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"DB_ID\", + \"PARAM_KEY\", + \"PARAM_VALUE\" +FROM + \"DATABASE_PARAMS\"" +); + +CREATE TABLE IF NOT EXISTS `DBS` ( + `DB_ID` bigint, + `DB_LOCATION_URI` string, + `NAME` string, + `OWNER_NAME` string, + `OWNER_TYPE` string, + CONSTRAINT `SYS_PK_DBS` PRIMARY KEY (`DB_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"DB_ID\", + \"DB_LOCATION_URI\", + \"NAME\", + \"OWNER_NAME\", + \"OWNER_TYPE\" +FROM + DBS" +); + +-- ANALYZE TABLE DBS COMPUTE STATISTICS; +-- ANALYZE TABLE DBS COMPUTE STATISTICS FOR COLUMNS; + +CREATE TABLE IF NOT EXISTS `DB_PRIVS` ( + `DB_GRANT_ID` bigint, + `CREATE_TIME` int, + `DB_ID` bigint, + `GRANT_OPTION` int, + `GRANTOR` string, + `GRANTOR_TYPE` string, + `PRINCIPAL_NAME` string, + `PRINCIPAL_TYPE` string, + `DB_PRIV` string, + CONSTRAINT `SYS_PK_DB_PRIVS` PRIMARY KEY (`DB_GRANT_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"DB_GRANT_ID\", + \"CREATE_TIME\", + \"DB_ID\", + \"GRANT_OPTION\", + \"GRANTOR\", + \"GRANTOR_TYPE\", + \"PRINCIPAL_NAME\", + \"PRINCIPAL_TYPE\", + \"DB_PRIV\" +FROM + \"DB_PRIVS\"" +); + +CREATE TABLE IF NOT EXISTS `GLOBAL_PRIVS` ( + `USER_GRANT_ID` bigint, + `CREATE_TIME` int, + `GRANT_OPTION` string, + `GRANTOR` string, + `GRANTOR_TYPE` string, + `PRINCIPAL_NAME` string, + `PRINCIPAL_TYPE` string, + `USER_PRIV` string, + CONSTRAINT `SYS_PK_GLOBAL_PRIVS` PRIMARY KEY (`USER_GRANT_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"USER_GRANT_ID\", + \"CREATE_TIME\", + \"GRANT_OPTION\", + \"GRANTOR\", + \"GRANTOR_TYPE\", + \"PRINCIPAL_NAME\", + \"PRINCIPAL_TYPE\", + \"USER_PRIV\" +FROM + \"GLOBAL_PRIVS\"" +); + +CREATE TABLE IF NOT EXISTS `IDXS` ( + `INDEX_ID` bigint, + `CREATE_TIME` int, + `DEFERRED_REBUILD` boolean, + `INDEX_HANDLER_CLASS` string, + `INDEX_NAME` string, + `INDEX_TBL_ID` bigint, + `LAST_ACCESS_TIME` int, + `ORIG_TBL_ID` bigint, + `SD_ID` bigint, + CONSTRAINT `SYS_PK_IDXS` PRIMARY KEY (`INDEX_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"INDEX_ID\", + \"CREATE_TIME\", + \"DEFERRED_REBUILD\", + \"INDEX_HANDLER_CLASS\", + \"INDEX_NAME\", + \"INDEX_TBL_ID\", + \"LAST_ACCESS_TIME\", + \"ORIG_TBL_ID\", + \"SD_ID\" +FROM + \"IDXS\"" +); + +CREATE TABLE IF NOT EXISTS `INDEX_PARAMS` ( + `INDEX_ID` bigint, + `PARAM_KEY` string, + `PARAM_VALUE` string, + CONSTRAINT `SYS_PK_INDEX_PARAMS` PRIMARY KEY (`INDEX_ID`,`PARAM_KEY`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"INDEX_ID\", + \"PARAM_KEY\", + \"PARAM_VALUE\" +FROM + \"INDEX_PARAMS\"" +); + +-- CREATE TABLE IF NOT EXISTS `NUCLEUS_TABLES` ( +-- `CLASS_NAME` string, +-- `TABLE_NAME` string, +-- `TYPE` string, +-- `OWNER` string, +-- `VERSION` string, +-- `INTERFACE_NAME` string, +-- CONSTRAINT `SYS_PK_NUCLEUS_TABLES` PRIMARY KEY (`CLASS_NAME`) DISABLE NOVALIDATE +-- ) +-- STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +-- TBLPROPERTIES ( +-- "hive.sql.database.type" = "METASTORE", +-- "hive.sql.query" = "SELECT * FROM NUCLEUS_TABLES" +-- ); + +CREATE TABLE IF NOT EXISTS `PARTITIONS` ( + `PART_ID` bigint, + `CREATE_TIME` int, + `LAST_ACCESS_TIME` int, + `PART_NAME` string, + `SD_ID` bigint, + `TBL_ID` bigint, + CONSTRAINT `SYS_PK_PARTITIONS` PRIMARY KEY (`PART_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"PART_ID\", + \"CREATE_TIME\", + \"LAST_ACCESS_TIME\", + \"PART_NAME\", + \"SD_ID\", + \"TBL_ID\" +FROM + \"PARTITIONS\"" +); + +-- CREATE TABLE IF NOT EXISTS `PARTITION_EVENTS` ( +-- `PART_NAME_ID` bigint, +-- `DB_NAME` string, +-- `EVENT_TIME` bigint, +-- `EVENT_TYPE` int, +-- `PARTITION_NAME` string, +-- `TBL_NAME` string, +-- CONSTRAINT `SYS_PK_PATITION_EVENTS` PRIMARY KEY (`PART_NAME_ID`) DISABLE NOVALIDATE +-- ) +-- STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +-- TBLPROPERTIES ( +-- "hive.sql.database.type" = "METASTORE", +-- "hive.sql.query" = "SELECT * FROM PARTITION_EVENTS" +-- ); + +CREATE TABLE IF NOT EXISTS `PARTITION_KEYS` ( + `TBL_ID` bigint, + `PKEY_COMMENT` string, + `PKEY_NAME` string, + `PKEY_TYPE` string, + `INTEGER_IDX` int, + CONSTRAINT `SYS_PK_PARTITION_KEYS` PRIMARY KEY (`TBL_ID`,`PKEY_NAME`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"TBL_ID\", + \"PKEY_COMMENT\", + \"PKEY_NAME\", + \"PKEY_TYPE\", + \"INTEGER_IDX\" +FROM + \"PARTITION_KEYS\"" +); + +CREATE TABLE IF NOT EXISTS `PARTITION_KEY_VALS` ( + `PART_ID` bigint, + `PART_KEY_VAL` string, + `INTEGER_IDX` int, + CONSTRAINT `SYS_PK_PARTITION_KEY_VALS` PRIMARY KEY (`PART_ID`,`INTEGER_IDX`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"PART_ID\", + \"PART_KEY_VAL\", + \"INTEGER_IDX\" +FROM + \"PARTITION_KEY_VALS\"" +); + +CREATE TABLE IF NOT EXISTS `PARTITION_PARAMS` ( + `PART_ID` bigint, + `PARAM_KEY` string, + `PARAM_VALUE` string, + CONSTRAINT `SYS_PK_PARTITION_PARAMS` PRIMARY KEY (`PART_ID`,`PARAM_KEY`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"PART_ID\", + \"PARAM_KEY\", + \"PARAM_VALUE\" +FROM + \"PARTITION_PARAMS\"" +); + +CREATE TABLE IF NOT EXISTS `PART_COL_PRIVS` ( + `PART_COLUMN_GRANT_ID` bigint, + `COLUMN_NAME` string, + `CREATE_TIME` int, + `GRANT_OPTION` int, + `GRANTOR` string, + `GRANTOR_TYPE` string, + `PART_ID` bigint, + `PRINCIPAL_NAME` string, + `PRINCIPAL_TYPE` string, + `PART_COL_PRIV` string, + CONSTRAINT `SYS_PK_PART_COL_PRIVS` PRIMARY KEY (`PART_COLUMN_GRANT_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"PART_COLUMN_GRANT_ID\", + \"COLUMN_NAME\", + \"CREATE_TIME\", + \"GRANT_OPTION\", + \"GRANTOR\", + \"GRANTOR_TYPE\", + \"PART_ID\", + \"PRINCIPAL_NAME\", + \"PRINCIPAL_TYPE\", + \"PART_COL_PRIV\" +FROM + \"PART_COL_PRIVS\"" +); + +CREATE TABLE IF NOT EXISTS `PART_PRIVS` ( + `PART_GRANT_ID` bigint, + `CREATE_TIME` int, + `GRANT_OPTION` int, + `GRANTOR` string, + `GRANTOR_TYPE` string, + `PART_ID` bigint, + `PRINCIPAL_NAME` string, + `PRINCIPAL_TYPE` string, + `PART_PRIV` string, + CONSTRAINT `SYS_PK_PART_PRIVS` PRIMARY KEY (`PART_GRANT_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"PART_GRANT_ID\", + \"CREATE_TIME\", + \"GRANT_OPTION\", + \"GRANTOR\", + \"GRANTOR_TYPE\", + \"PART_ID\", + \"PRINCIPAL_NAME\", + \"PRINCIPAL_TYPE\", + \"PART_PRIV\" +FROM + \"PART_PRIVS\"" +); + +CREATE TABLE IF NOT EXISTS `ROLES` ( + `ROLE_ID` bigint, + `CREATE_TIME` int, + `OWNER_NAME` string, + `ROLE_NAME` string, + CONSTRAINT `SYS_PK_ROLES` PRIMARY KEY (`ROLE_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"ROLE_ID\", + \"CREATE_TIME\", + \"OWNER_NAME\", + \"ROLE_NAME\" +FROM + \"ROLES\"" +); + +CREATE TABLE IF NOT EXISTS `ROLE_MAP` ( + `ROLE_GRANT_ID` bigint, + `ADD_TIME` int, + `GRANT_OPTION` int, + `GRANTOR` string, + `GRANTOR_TYPE` string, + `PRINCIPAL_NAME` string, + `PRINCIPAL_TYPE` string, + `ROLE_ID` bigint, + CONSTRAINT `SYS_PK_ROLE_MAP` PRIMARY KEY (`ROLE_GRANT_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"ROLE_GRANT_ID\", + \"ADD_TIME\", + \"GRANT_OPTION\", + \"GRANTOR\", + \"GRANTOR_TYPE\", + \"PRINCIPAL_NAME\", + \"PRINCIPAL_TYPE\", + \"ROLE_ID\" +FROM + \"ROLE_MAP\"" +); + +CREATE TABLE IF NOT EXISTS `SDS` ( + `SD_ID` bigint, + `CD_ID` bigint, + `INPUT_FORMAT` string, + `IS_COMPRESSED` boolean, + `IS_STOREDASSUBDIRECTORIES` boolean, + `LOCATION` string, + `NUM_BUCKETS` int, + `OUTPUT_FORMAT` string, + `SERDE_ID` bigint, + CONSTRAINT `SYS_PK_SDS` PRIMARY KEY (`SD_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"SD_ID\", + \"CD_ID\", + \"INPUT_FORMAT\", + \"IS_COMPRESSED\", + \"IS_STOREDASSUBDIRECTORIES\", + \"LOCATION\", + \"NUM_BUCKETS\", + \"OUTPUT_FORMAT\", + \"SERDE_ID\" +FROM + \"SDS\"" +); + +CREATE TABLE IF NOT EXISTS `SD_PARAMS` ( + `SD_ID` bigint, + `PARAM_KEY` string, + `PARAM_VALUE` string, + CONSTRAINT `SYS_PK_SD_PARAMS` PRIMARY KEY (`SD_ID`,`PARAM_KEY`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"SD_ID\", + \"PARAM_KEY\", + \"PARAM_VALUE\" +FROM + \"SD_PARAMS\"" +); + +CREATE TABLE IF NOT EXISTS `SEQUENCE_TABLE` ( + `SEQUENCE_NAME` string, + `NEXT_VAL` bigint, + CONSTRAINT `SYS_PK_SEQUENCE_TABLE` PRIMARY KEY (`SEQUENCE_NAME`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"SEQUENCE_NAME\", + \"NEXT_VAL\" +FROM + \"SEQUENCE_TABLE\"" +); + +CREATE TABLE IF NOT EXISTS `SERDES` ( + `SERDE_ID` bigint, + `NAME` string, + `SLIB` string, + CONSTRAINT `SYS_PK_SERDES` PRIMARY KEY (`SERDE_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"SERDE_ID\", + \"NAME\", + \"SLIB\" +FROM + \"SERDES\"" +); + +CREATE TABLE IF NOT EXISTS `SERDE_PARAMS` ( + `SERDE_ID` bigint, + `PARAM_KEY` string, + `PARAM_VALUE` string, + CONSTRAINT `SYS_PK_SERDE_PARAMS` PRIMARY KEY (`SERDE_ID`,`PARAM_KEY`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"SERDE_ID\", + \"PARAM_KEY\", + \"PARAM_VALUE\" +FROM + \"SERDE_PARAMS\"" +); + +CREATE TABLE IF NOT EXISTS `SKEWED_COL_NAMES` ( + `SD_ID` bigint, + `SKEWED_COL_NAME` string, + `INTEGER_IDX` int, + CONSTRAINT `SYS_PK_SKEWED_COL_NAMES` PRIMARY KEY (`SD_ID`,`INTEGER_IDX`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"SD_ID\", + \"SKEWED_COL_NAME\", + \"INTEGER_IDX\" +FROM + \"SKEWED_COL_NAMES\"" +); + +CREATE TABLE IF NOT EXISTS `SKEWED_COL_VALUE_LOC_MAP` ( + `SD_ID` bigint, + `STRING_LIST_ID_KID` bigint, + `LOCATION` string, + CONSTRAINT `SYS_PK_COL_VALUE_LOC_MAP` PRIMARY KEY (`SD_ID`,`STRING_LIST_ID_KID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"SD_ID\", + \"STRING_LIST_ID_KID\", + \"LOCATION\" +FROM + \"SKEWED_COL_VALUE_LOC_MAP\"" +); + +CREATE TABLE IF NOT EXISTS `SKEWED_STRING_LIST` ( + `STRING_LIST_ID` bigint, + CONSTRAINT `SYS_PK_SKEWED_STRING_LIST` PRIMARY KEY (`STRING_LIST_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"STRING_LIST_ID\" +FROM + \"SKEWED_STRING_LIST\"" +); + +CREATE TABLE IF NOT EXISTS `SKEWED_STRING_LIST_VALUES` ( + `STRING_LIST_ID` bigint, + `STRING_LIST_VALUE` string, + `INTEGER_IDX` int, + CONSTRAINT `SYS_PK_SKEWED_STRING_LIST_VALUES` PRIMARY KEY (`STRING_LIST_ID`,`INTEGER_IDX`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"STRING_LIST_ID\", + \"STRING_LIST_VALUE\", + \"INTEGER_IDX\" +FROM + \"SKEWED_STRING_LIST_VALUES\"" +); + +CREATE TABLE IF NOT EXISTS `SKEWED_VALUES` ( + `SD_ID_OID` bigint, + `STRING_LIST_ID_EID` bigint, + `INTEGER_IDX` int, + CONSTRAINT `SYS_PK_SKEWED_VALUES` PRIMARY KEY (`SD_ID_OID`,`INTEGER_IDX`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"SD_ID_OID\", + \"STRING_LIST_ID_EID\", + \"INTEGER_IDX\" +FROM + \"SKEWED_VALUES\"" +); + +CREATE TABLE IF NOT EXISTS `SORT_COLS` ( + `SD_ID` bigint, + `COLUMN_NAME` string, + `ORDER` int, + `INTEGER_IDX` int, + CONSTRAINT `SYS_PK_SORT_COLS` PRIMARY KEY (`SD_ID`,`INTEGER_IDX`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"SD_ID\", + \"COLUMN_NAME\", + \"ORDER\", + \"INTEGER_IDX\" +FROM + \"SORT_COLS\"" +); + +CREATE TABLE IF NOT EXISTS `TABLE_PARAMS` ( + `TBL_ID` bigint, + `PARAM_KEY` string, + `PARAM_VALUE` string, + CONSTRAINT `SYS_PK_TABLE_PARAMS` PRIMARY KEY (`TBL_ID`,`PARAM_KEY`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"TBL_ID\", + \"PARAM_KEY\", + \"PARAM_VALUE\" +FROM + \"TABLE_PARAMS\"" +); + +CREATE TABLE IF NOT EXISTS `TBLS` ( + `TBL_ID` bigint, + `CREATE_TIME` int, + `DB_ID` bigint, + `LAST_ACCESS_TIME` int, + `OWNER` string, + `RETENTION` int, + `SD_ID` bigint, + `TBL_NAME` string, + `TBL_TYPE` string, + `VIEW_EXPANDED_TEXT` string, + `VIEW_ORIGINAL_TEXT` string, + `IS_REWRITE_ENABLED` boolean, + CONSTRAINT `SYS_PK_TBLS` PRIMARY KEY (`TBL_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"TBL_ID\", + \"CREATE_TIME\", + \"DB_ID\", + \"LAST_ACCESS_TIME\", + \"OWNER\", + \"RETENTION\", + \"SD_ID\", + \"TBL_NAME\", + \"TBL_TYPE\", + \"VIEW_EXPANDED_TEXT\", + \"VIEW_ORIGINAL_TEXT\", + \"IS_REWRITE_ENABLED\" +FROM TBLS" +); + +CREATE TABLE IF NOT EXISTS `TBL_COL_PRIVS` ( + `TBL_COLUMN_GRANT_ID` bigint, + `COLUMN_NAME` string, + `CREATE_TIME` int, + `GRANT_OPTION` int, + `GRANTOR` string, + `GRANTOR_TYPE` string, + `PRINCIPAL_NAME` string, + `PRINCIPAL_TYPE` string, + `TBL_COL_PRIV` string, + `TBL_ID` bigint, + CONSTRAINT `SYS_PK_TBL_COL_PRIVS` PRIMARY KEY (`TBL_COLUMN_GRANT_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"TBL_COLUMN_GRANT_ID\", + \"COLUMN_NAME\", + \"CREATE_TIME\", + \"GRANT_OPTION\", + \"GRANTOR\", + \"GRANTOR_TYPE\", + \"PRINCIPAL_NAME\", + \"PRINCIPAL_TYPE\", + \"TBL_COL_PRIV\", + \"TBL_ID\" +FROM + \"TBL_COL_PRIVS\"" +); + +CREATE TABLE IF NOT EXISTS `TBL_PRIVS` ( + `TBL_GRANT_ID` bigint, + `CREATE_TIME` int, + `GRANT_OPTION` int, + `GRANTOR` string, + `GRANTOR_TYPE` string, + `PRINCIPAL_NAME` string, + `PRINCIPAL_TYPE` string, + `TBL_PRIV` string, + `TBL_ID` bigint, + CONSTRAINT `SYS_PK_TBL_PRIVS` PRIMARY KEY (`TBL_GRANT_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"TBL_GRANT_ID\", + \"CREATE_TIME\", + \"GRANT_OPTION\", + \"GRANTOR\", + \"GRANTOR_TYPE\", + \"PRINCIPAL_NAME\", + \"PRINCIPAL_TYPE\", + \"TBL_PRIV\", + \"TBL_ID\" +FROM + \"TBL_PRIVS\"" +); + +CREATE TABLE IF NOT EXISTS `TAB_COL_STATS` ( + `CS_ID` bigint, + `DB_NAME` string, + `TABLE_NAME` string, + `COLUMN_NAME` string, + `COLUMN_TYPE` string, + `TBL_ID` bigint, + `LONG_LOW_VALUE` bigint, + `LONG_HIGH_VALUE` bigint, + `DOUBLE_HIGH_VALUE` double, + `DOUBLE_LOW_VALUE` double, + `BIG_DECIMAL_LOW_VALUE` string, + `BIG_DECIMAL_HIGH_VALUE` string, + `NUM_NULLS` bigint, + `NUM_DISTINCTS` bigint, + `AVG_COL_LEN` double, + `MAX_COL_LEN` bigint, + `NUM_TRUES` bigint, + `NUM_FALSES` bigint, + `LAST_ANALYZED` bigint, + CONSTRAINT `SYS_PK_TAB_COL_STATS` PRIMARY KEY (`CS_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"CS_ID\", + \"DB_NAME\", + \"TABLE_NAME\", + \"COLUMN_NAME\", + \"COLUMN_TYPE\", + \"TBL_ID\", + \"LONG_LOW_VALUE\", + \"LONG_HIGH_VALUE\", + \"DOUBLE_HIGH_VALUE\", + \"DOUBLE_LOW_VALUE\", + \"BIG_DECIMAL_LOW_VALUE\", + \"BIG_DECIMAL_HIGH_VALUE\", + \"NUM_NULLS\", + \"NUM_DISTINCTS\", + \"AVG_COL_LEN\", + \"MAX_COL_LEN\", + \"NUM_TRUES\", + \"NUM_FALSES\", + \"LAST_ANALYZED\" +FROM + \"TAB_COL_STATS\"" +); + +CREATE TABLE IF NOT EXISTS `PART_COL_STATS` ( + `CS_ID` bigint, + `DB_NAME` string, + `TABLE_NAME` string, + `PARTITION_NAME` string, + `COLUMN_NAME` string, + `COLUMN_TYPE` string, + `PART_ID` bigint, + `LONG_LOW_VALUE` bigint, + `LONG_HIGH_VALUE` bigint, + `DOUBLE_HIGH_VALUE` double, + `DOUBLE_LOW_VALUE` double, + `BIG_DECIMAL_LOW_VALUE` string, + `BIG_DECIMAL_HIGH_VALUE` string, + `NUM_NULLS` bigint, + `NUM_DISTINCTS` bigint, + `AVG_COL_LEN` double, + `MAX_COL_LEN` bigint, + `NUM_TRUES` bigint, + `NUM_FALSES` bigint, + `LAST_ANALYZED` bigint, + CONSTRAINT `SYS_PK_PART_COL_STATS` PRIMARY KEY (`CS_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"CS_ID\", + \"DB_NAME\", + \"TABLE_NAME\", + \"PARTITION_NAME\", + \"COLUMN_NAME\", + \"COLUMN_TYPE\", + \"PART_ID\", + \"LONG_LOW_VALUE\", + \"LONG_HIGH_VALUE\", + \"DOUBLE_HIGH_VALUE\", + \"DOUBLE_LOW_VALUE\", + \"BIG_DECIMAL_LOW_VALUE\", + \"BIG_DECIMAL_HIGH_VALUE\", + \"NUM_NULLS\", + \"NUM_DISTINCTS\", + \"AVG_COL_LEN\", + \"MAX_COL_LEN\", + \"NUM_TRUES\", + \"NUM_FALSES\", + \"LAST_ANALYZED\" +FROM + \"PART_COL_STATS\"" +); + +-- CREATE TABLE IF NOT EXISTS `TYPES` ( +-- `TYPES_ID` bigint, +-- `TYPE_NAME` string, +-- `TYPE1` string, +-- `TYPE2` string, +-- CONSTRAINT `SYS_PK_TYPES` PRIMARY KEY (`TYPES_ID`) DISABLE NOVALIDATE +-- ) +-- STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +-- TBLPROPERTIES ( +-- "hive.sql.database.type" = "METASTORE", +-- "hive.sql.query" = "SELECT * FROM TYPES" +-- ); + +-- CREATE TABLE IF NOT EXISTS `TYPE_FIELDS` ( +-- `TYPE_NAME` bigint, +-- `COMMENT` string, +-- `FIELD_NAME` string, +-- `FIELD_TYPE` string, +-- `INTEGER_IDX` int, +-- CONSTRAINT `SYS_PK_TYPE_FIELDS` PRIMARY KEY (`TYPE_NAME`,`FIELD_NAME`) DISABLE NOVALIDATE +-- ) +-- STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +-- TBLPROPERTIES ( +-- "hive.sql.database.type" = "METASTORE", +-- "hive.sql.query" = "SELECT * FROM TYPE_FIELDS" +-- ); + +-- CREATE TABLE IF NOT EXISTS `MASTER_KEYS` +-- ( +-- `KEY_ID` INTEGER, +-- `MASTER_KEY` binary, +-- CONSTRAINT `SYS_PK_MASTER_KEYS` PRIMARY KEY (`KEY_ID`) DISABLE NOVALIDATE +-- ) +-- STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +-- TBLPROPERTIES ( +-- "hive.sql.database.type" = "METASTORE", +-- "hive.sql.query" = "SELECT * FROM MASTER_KEYS" +-- ); + +-- CREATE TABLE IF NOT EXISTS `DELEGATION_TOKENS` +-- ( +-- `TOKEN_IDENT` string, +-- `TOKEN` string, +-- CONSTRAINT `SYS_PK_DELEGATION_TOKENS` PRIMARY KEY (`TOKEN_IDENT`) DISABLE NOVALIDATE +-- ) +-- STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +-- TBLPROPERTIES ( +-- "hive.sql.database.type" = "METASTORE", +-- "hive.sql.query" = "SELECT * FROM DELEGATION_TOKENS" +-- ); + +CREATE TABLE IF NOT EXISTS `VERSION` ( + `VER_ID` BIGINT, + `SCHEMA_VERSION` string, + `VERSION_COMMENT` string, + CONSTRAINT `SYS_PK_VERSION` PRIMARY KEY (`VER_ID`) DISABLE NOVALIDATE +); + +CREATE TABLE IF NOT EXISTS `DB_VERSION` ( + `VER_ID` BIGINT, + `SCHEMA_VERSION` string, + `VERSION_COMMENT` string, + CONSTRAINT `SYS_PK_DB_VERSION` PRIMARY KEY (`VER_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"VER_ID\", + \"SCHEMA_VERSION\", + \"VERSION_COMMENT\" +FROM + \"VERSION\"" +); + +CREATE TABLE IF NOT EXISTS `FUNCS` ( + `FUNC_ID` bigint, + `CLASS_NAME` string, + `CREATE_TIME` int, + `DB_ID` bigint, + `FUNC_NAME` string, + `FUNC_TYPE` int, + `OWNER_NAME` string, + `OWNER_TYPE` string, + CONSTRAINT `SYS_PK_FUNCS` PRIMARY KEY (`FUNC_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"FUNC_ID\", + \"CLASS_NAME\", + \"CREATE_TIME\", + \"DB_ID\", + \"FUNC_NAME\", + \"FUNC_TYPE\", + \"OWNER_NAME\", + \"OWNER_TYPE\" +FROM + \"FUNCS\"" +); + +-- CREATE TABLE IF NOT EXISTS `FUNC_RU` ( +-- `FUNC_ID` bigint, +-- `RESOURCE_TYPE` int, +-- `RESOURCE_URI` string, +-- `INTEGER_IDX` int, +-- CONSTRAINT `SYS_PK_FUNCS_RU` PRIMARY KEY (`FUNC_ID`, `INTEGER_IDX`) DISABLE NOVALIDATE +-- ) +-- STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +-- TBLPROPERTIES ( +-- "hive.sql.database.type" = "METASTORE", +-- "hive.sql.query" = "SELECT * FROM FUNCS_RU" +-- ); + +-- CREATE TABLE IF NOT EXISTS `NOTIFICATION_LOG` +-- ( +-- `NL_ID` bigint, +-- `EVENT_ID` bigint, +-- `EVENT_TIME` int, +-- `EVENT_TYPE` string, +-- `DB_NAME` string, +-- `TBL_NAME` string, +-- `MESSAGE` string, +-- CONSTRAINT `SYS_PK_NOTIFICATION_LOG` PRIMARY KEY (`NL_ID`) DISABLE NOVALIDATE +-- ) +-- STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +-- TBLPROPERTIES ( +-- "hive.sql.database.type" = "METASTORE", +-- "hive.sql.query" = "SELECT * FROM NOTIFICATION_LOG" +-- ); + +-- CREATE TABLE IF NOT EXISTS `NOTIFICATION_SEQUENCE` +-- ( +-- `NNI_ID` bigint, +-- `NEXT_EVENT_ID` bigint, +-- CONSTRAINT `SYS_PK_NOTIFICATION_SEQUENCE` PRIMARY KEY (`NNI_ID`) DISABLE NOVALIDATE +-- ) +-- STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +-- TBLPROPERTIES ( +-- "hive.sql.database.type" = "METASTORE", +-- "hive.sql.query" = "SELECT * FROM NOTIFICATION_SEQUENCE" +-- ); + +CREATE TABLE IF NOT EXISTS `KEY_CONSTRAINTS` +( + `CHILD_CD_ID` bigint, + `CHILD_INTEGER_IDX` int, + `CHILD_TBL_ID` bigint, + `PARENT_CD_ID` bigint, + `PARENT_INTEGER_IDX` int, + `PARENT_TBL_ID` bigint, + `POSITION` bigint, + `CONSTRAINT_NAME` string, + `CONSTRAINT_TYPE` string, + `UPDATE_RULE` string, + `DELETE_RULE` string, + `ENABLE_VALIDATE_RELY` int, + CONSTRAINT `SYS_PK_KEY_CONSTRAINTS` PRIMARY KEY (`CONSTRAINT_NAME`, `POSITION`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"CHILD_CD_ID\", + \"CHILD_INTEGER_IDX\", + \"CHILD_TBL_ID\", + \"PARENT_CD_ID\", + \"PARENT_INTEGER_IDX\", + \"PARENT_TBL_ID\", + \"POSITION\", + \"CONSTRAINT_NAME\", + \"CONSTRAINT_TYPE\", + \"UPDATE_RULE\", + \"DELETE_RULE\", + \"ENABLE_VALIDATE_RELY\" +FROM + \"KEY_CONSTRAINTS\"" +); + +CREATE DATABASE INFORMATION_SCHEMA; + +USE INFORMATION_SCHEMA; + +CREATE VIEW IF NOT EXISTS `SCHEMATA` +( + `CATALOG_NAME`, + `SCHEMA_NAME`, + `SCHEMA_OWNER`, + `DEFAULT_CHARACTER_SET_CATALOG`, + `DEFAULT_CHARACTER_SET_SCHEMA`, + `DEFAULT_CHARACTER_SET_NAME`, + `SQL_PATH` +) AS +SELECT + 'default', + `NAME`, + `OWNER_NAME`, + cast(null as string), + cast(null as string), + cast(null as string), + `DB_LOCATION_URI` +FROM + sys.DBS; + +CREATE VIEW IF NOT EXISTS `TABLES` +( + `TABLE_CATALOG`, + `TABLE_SCHEMA`, + `TABLE_NAME`, + `TABLE_TYPE`, + `SELF_REFERENCING_COLUMN_NAME`, + `REFERENCE_GENERATION`, + `USER_DEFINED_TYPE_CATALOG`, + `USER_DEFINED_TYPE_SCHEMA`, + `USER_DEFINED_TYPE_NAME`, + `IS_INSERTABLE_INTO`, + `IS_TYPED`, + `COMMIT_ACTION` +) AS +SELECT + 'default', + D.NAME, + T.TBL_NAME, + IF(length(T.VIEW_ORIGINAL_TEXT) > 0, 'VIEW', 'BASE_TABLE'), + cast(null as string), + cast(null as string), + cast(null as string), + cast(null as string), + cast(null as string), + IF(length(T.VIEW_ORIGINAL_TEXT) > 0, 'NO', 'YES'), + 'NO', + cast(null as string) +FROM + `sys`.`TBLS` T, `sys`.`DBS` D +WHERE + D.`DB_ID` = T.`DB_ID`; + +CREATE VIEW IF NOT EXISTS `TABLE_PRIVILEGES` +( + `GRANTOR`, + `GRANTEE`, + `TABLE_CATALOG`, + `TABLE_SCHEMA`, + `TABLE_NAME`, + `PRIVILEGE_TYPE`, + `IS_GRANTABLE`, + `WITH_HIERARCHY` +) AS +SELECT + `GRANTOR`, + `PRINCIPAL_NAME`, + 'default', + D.`NAME`, + T.`TBL_NAME`, + P.`TBL_PRIV`, + IF (P.`GRANT_OPTION` == 0, 'NO', 'YES'), + 'NO' +FROM + sys.`TBL_PRIVS` P, + sys.`TBLS` T, + sys.`DBS` D +WHERE + P.TBL_ID = T.TBL_ID + AND T.DB_ID = D.DB_ID; + +CREATE VIEW IF NOT EXISTS `COLUMNS` +( + `TABLE_CATALOG`, + `TABLE_SCHEMA`, + `TABLE_NAME`, + `COLUMN_NAME`, + `ORDINAL_POSITION`, + `COLUMN_DEFAULT`, + `IS_NULLABLE`, + `DATA_TYPE`, + `CHARACTER_MAXIMUM_LENGTH`, + `CHARACTER_OCTET_LENGTH`, + `NUMERIC_PRECISION`, + `NUMERIC_PRECISION_RADIX`, + `NUMERIC_SCALE`, + `DATETIME_PRECISION`, + `INTERVAL_TYPE`, + `INTERVAL_PRECISION`, + `CHARACTER_SET_CATALOG`, + `CHARACTER_SET_SCHEMA`, + `CHARACTER_SET_NAME`, + `COLLATION_CATALOG`, + `COLLATION_SCHEMA`, + `COLLATION_NAME`, + `UDT_CATALOG`, + `UDT_SCHEMA`, + `UDT_NAME`, + `SCOPE_CATALOG`, + `SCOPE_SCHEMA`, + `SCOPE_NAME`, + `MAXIMUM_CARDINALITY`, + `DTD_IDENTIFIER`, + `IS_SELF_REFERENCING`, + `IS_IDENTITY`, + `IDENTITY_GENERATION`, + `IDENTITY_START`, + `IDENTITY_INCREMENT`, + `IDENTITY_MAXIMUM`, + `IDENTITY_MINIMUM`, + `IDENTITY_CYCLE`, + `IS_GENERATED`, + `GENERATION_EXPRESSION`, + `IS_SYSTEM_TIME_PERIOD_START`, + `IS_SYSTEM_TIME_PERIOD_END`, + `SYSTEM_TIME_PERIOD_TIMESTAMP_GENERATION`, + `IS_UPDATABLE`, + `DECLARED_DATA_TYPE`, + `DECLARED_NUMERIC_PRECISION`, + `DECLARED_NUMERIC_SCALE` +) AS +SELECT + 'default', + D.NAME, + T.TBL_NAME, + C.COLUMN_NAME, + C.INTEGER_IDX, + cast (null as string), + 'YES', + C.TYPE_NAME as TYPE_NAME, + CASE WHEN lower(C.TYPE_NAME) like 'varchar%' THEN cast(regexp_extract(upper(C.TYPE_NAME), '^VARCHAR\\s*\\((\\d+)\\s*\\)$', 1) as int) + WHEN lower(C.TYPE_NAME) like 'char%' THEN cast(regexp_extract(upper(C.TYPE_NAME), '^CHAR\\s*\\((\\d+)\\s*\\)$', 1) as int) + ELSE null END, + CASE WHEN lower(C.TYPE_NAME) like 'varchar%' THEN cast(regexp_extract(upper(C.TYPE_NAME), '^VARCHAR\\s*\\((\\d+)\\s*\\)$', 1) as int) + WHEN lower(C.TYPE_NAME) like 'char%' THEN cast(regexp_extract(upper(C.TYPE_NAME), '^CHAR\\s*\\((\\d+)\\s*\\)$', 1) as int) + ELSE null END, + CASE WHEN lower(C.TYPE_NAME) = 'bigint' THEN 19 + WHEN lower(C.TYPE_NAME) = 'int' THEN 10 + WHEN lower(C.TYPE_NAME) = 'smallint' THEN 5 + WHEN lower(C.TYPE_NAME) = 'tinyint' THEN 3 + WHEN lower(C.TYPE_NAME) = 'float' THEN 23 + WHEN lower(C.TYPE_NAME) = 'double' THEN 53 + WHEN lower(C.TYPE_NAME) like 'decimal%' THEN regexp_extract(upper(C.TYPE_NAME), '^DECIMAL\\s*\\((\\d+)',1) + WHEN lower(C.TYPE_NAME) like 'numeric%' THEN regexp_extract(upper(C.TYPE_NAME), '^NUMERIC\\s*\\((\\d+)',1) + ELSE null END, + CASE WHEN lower(C.TYPE_NAME) = 'bigint' THEN 10 + WHEN lower(C.TYPE_NAME) = 'int' THEN 10 + WHEN lower(C.TYPE_NAME) = 'smallint' THEN 10 + WHEN lower(C.TYPE_NAME) = 'tinyint' THEN 10 + WHEN lower(C.TYPE_NAME) = 'float' THEN 2 + WHEN lower(C.TYPE_NAME) = 'double' THEN 2 + WHEN lower(C.TYPE_NAME) like 'decimal%' THEN 10 + WHEN lower(C.TYPE_NAME) like 'numeric%' THEN 10 + ELSE null END, + CASE WHEN lower(C.TYPE_NAME) like 'decimal%' THEN regexp_extract(upper(C.TYPE_NAME), '^DECIMAL\\s*\\((\\d+),(\\d+)',2) + WHEN lower(C.TYPE_NAME) like 'numeric%' THEN regexp_extract(upper(C.TYPE_NAME), '^NUMERIC\\s*\\((\\d+),(\\d+)',2) + ELSE null END, + CASE WHEN lower(C.TYPE_NAME) = 'date' THEN 0 + WHEN lower(C.TYPE_NAME) = 'timestamp' THEN 9 + ELSE null END, + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + C.CD_ID, + 'NO', + 'NO', + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + 'NEVER', + cast (null as string), + 'NO', + 'NO', + cast (null as string), + 'YES', + C.TYPE_NAME as DECLARED_DATA_TYPE, + CASE WHEN lower(C.TYPE_NAME) = 'bigint' THEN 19 + WHEN lower(C.TYPE_NAME) = 'int' THEN 10 + WHEN lower(C.TYPE_NAME) = 'smallint' THEN 5 + WHEN lower(C.TYPE_NAME) = 'tinyint' THEN 3 + WHEN lower(C.TYPE_NAME) = 'float' THEN 23 + WHEN lower(C.TYPE_NAME) = 'double' THEN 53 + WHEN lower(C.TYPE_NAME) like 'decimal%' THEN regexp_extract(upper(C.TYPE_NAME), '^DECIMAL\\s*\\((\\d+)',1) + WHEN lower(C.TYPE_NAME) like 'numeric%' THEN regexp_extract(upper(C.TYPE_NAME), '^NUMERIC\\s*\\((\\d+)',1) + ELSE null END, + CASE WHEN lower(C.TYPE_NAME) = 'bigint' THEN 10 + WHEN lower(C.TYPE_NAME) = 'int' THEN 10 + WHEN lower(C.TYPE_NAME) = 'smallint' THEN 10 + WHEN lower(C.TYPE_NAME) = 'tinyint' THEN 10 + WHEN lower(C.TYPE_NAME) = 'float' THEN 2 + WHEN lower(C.TYPE_NAME) = 'double' THEN 2 + WHEN lower(C.TYPE_NAME) like 'decimal%' THEN 10 + WHEN lower(C.TYPE_NAME) like 'numeric%' THEN 10 + ELSE null END +FROM + sys.`COLUMNS_V2` C, + sys.`SDS` S, + sys.`TBLS` T, + sys.`DBS` D +WHERE + S.`SD_ID` = T.`SD_ID` + AND T.`DB_ID` = D.`DB_ID` + AND C.`CD_ID` = S.`CD_ID`; + +CREATE VIEW IF NOT EXISTS `COLUMN_PRIVILEGES` +( + `GRANTOR`, + `GRANTEE`, + `TABLE_CATALOG`, + `TABLE_SCHEMA`, + `TABLE_NAME`, + `COLUMN_NAME`, + `PRIVILEGE_TYPE`, + `IS_GRANTABLE` +) AS +SELECT + `GRANTOR`, + `PRINCIPAL_NAME`, + 'default', + D.`NAME`, + T.`TBL_NAME`, + C.`COLUMN_NAME`, + P.`TBL_COL_PRIV`, + IF (P.`GRANT_OPTION` == 0, 'NO', 'YES') +FROM + sys.`TBL_COL_PRIVS` P, + sys.`TBLS` T, + sys.`DBS` D, + sys.`COLUMNS_V2` C, + sys.`SDS` S +WHERE + S.`SD_ID` = T.`SD_ID` + AND T.`DB_ID` = D.`DB_ID` + AND P.`TBL_ID` = T.`TBL_ID` + AND P.`COLUMN_NAME` = C.`COLUMN_NAME` + AND C.`CD_ID` = S.`CD_ID`; + +CREATE VIEW IF NOT EXISTS `VIEWS` +( + `TABLE_CATALOG`, + `TABLE_SCHEMA`, + `TABLE_NAME`, + `VIEW_DEFINITION`, + `CHECK_OPTION`, + `IS_UPDATABLE`, + `IS_INSERTABLE_INTO`, + `IS_TRIGGER_UPDATABLE`, + `IS_TRIGGER_DELETABLE`, + `IS_TRIGGER_INSERTABLE_INTO` +) AS +SELECT + 'default', + D.NAME, + T.TBL_NAME, + T.VIEW_ORIGINAL_TEXT, + CAST(NULL as string), + false, + false, + false, + false, + false +FROM + `sys`.`DBS` D, + `sys`.`TBLS` T +WHERE + D.`DB_ID` = T.`DB_ID` AND + length(T.VIEW_ORIGINAL_TEXT) > 0; diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java index 4d727ba..f3c571a 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/FileSinkOperator.java @@ -357,8 +357,16 @@ protected void initializeOp(Configuration hconf) throws HiveException { taskId = Utilities.getTaskId(hconf); initializeSpecPath(); fs = specPath.getFileSystem(hconf); + + if (hconf instanceof JobConf) { + jc = (JobConf) hconf; + } else { + // test code path + jc = new JobConf(hconf); + } + try { - createHiveOutputFormat(hconf); + createHiveOutputFormat(jc); } catch (HiveException ex) { logOutputFormatError(hconf, ex); throw ex; @@ -379,12 +387,6 @@ protected void initializeOp(Configuration hconf) throws HiveException { // half of the script.timeout but less than script.timeout, we will still // be able to report progress. timeOut = hconf.getInt("mapred.healthChecker.script.timeout", 600000) / 2; - if (hconf instanceof JobConf) { - jc = (JobConf) hconf; - } else { - // test code path - jc = new JobConf(hconf); - } if (multiFileSpray) { partitionEval = new ExprNodeEvaluator[conf.getPartitionCols().size()]; @@ -1158,12 +1160,12 @@ public void checkOutputSpecs(FileSystem ignored, JobConf job) throws IOException } } - private void createHiveOutputFormat(Configuration hconf) throws HiveException { + private void createHiveOutputFormat(JobConf job) throws HiveException { if (hiveOutputFormat == null) { - Utilities.copyTableJobPropertiesToConf(conf.getTableInfo(), hconf); + Utilities.copyTableJobPropertiesToConf(conf.getTableInfo(), job); } try { - hiveOutputFormat = HiveFileFormatUtils.getHiveOutputFormat(hconf, getConf().getTableInfo()); + hiveOutputFormat = HiveFileFormatUtils.getHiveOutputFormat(job, getConf().getTableInfo()); } catch (Throwable t) { throw (t instanceof HiveException) ? (HiveException)t : new HiveException(t); } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java index 9036d9e..ebf1344 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java @@ -145,6 +145,8 @@ import org.apache.hadoop.mapred.Reporter; import org.apache.hadoop.mapred.SequenceFileInputFormat; import org.apache.hadoop.mapred.SequenceFileOutputFormat; +import org.apache.hadoop.security.Credentials; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.Progressable; import org.apache.hive.common.util.ACLConfigurationParser; import org.apache.hive.common.util.ReflectionUtil; @@ -2027,7 +2029,7 @@ public static int getDefaultNotificationInterval(Configuration hconf) { * @param job * configuration which receives configured properties */ - public static void copyTableJobPropertiesToConf(TableDesc tbl, Configuration job) { + public static void copyTableJobPropertiesToConf(TableDesc tbl, JobConf job) throws HiveException { Properties tblProperties = tbl.getProperties(); for(String name: tblProperties.stringPropertyNames()) { if (job.get(name) == null) { @@ -2038,11 +2040,23 @@ public static void copyTableJobPropertiesToConf(TableDesc tbl, Configuration job } } Map jobProperties = tbl.getJobProperties(); - if (jobProperties == null) { - return; + if (jobProperties != null) { + for (Map.Entry entry : jobProperties.entrySet()) { + job.set(entry.getKey(), entry.getValue()); + } } - for (Map.Entry entry : jobProperties.entrySet()) { - job.set(entry.getKey(), entry.getValue()); + + try { + Map jobSecrets = tbl.getJobSecrets(); + if (jobSecrets != null) { + for (Map.Entry entry : jobSecrets.entrySet()) { + job.getCredentials().addSecretKey(new Text(entry.getKey()), entry.getValue().getBytes()); + UserGroupInformation.getCurrentUser().getCredentials() + .addSecretKey(new Text(entry.getKey()), entry.getValue().getBytes()); + } + } + } catch (IOException e) { + throw new HiveException(e); } } @@ -2055,7 +2069,7 @@ public static void copyTableJobPropertiesToConf(TableDesc tbl, Configuration job * @param tbl * @param job */ - public static void copyTablePropertiesToConf(TableDesc tbl, JobConf job) { + public static void copyTablePropertiesToConf(TableDesc tbl, JobConf job) throws HiveException { Properties tblProperties = tbl.getProperties(); for(String name: tblProperties.stringPropertyNames()) { String val = (String) tblProperties.get(name); @@ -2064,11 +2078,23 @@ public static void copyTablePropertiesToConf(TableDesc tbl, JobConf job) { } } Map jobProperties = tbl.getJobProperties(); - if (jobProperties == null) { - return; + if (jobProperties != null) { + for (Map.Entry entry : jobProperties.entrySet()) { + job.set(entry.getKey(), entry.getValue()); + } } - for (Map.Entry entry : jobProperties.entrySet()) { - job.set(entry.getKey(), entry.getValue()); + + try { + Map jobSecrets = tbl.getJobSecrets(); + if (jobSecrets != null) { + for (Map.Entry entry : jobSecrets.entrySet()) { + job.getCredentials().addSecretKey(new Text(entry.getKey()), entry.getValue().getBytes()); + UserGroupInformation.getCurrentUser().getCredentials() + .addSecretKey(new Text(entry.getKey()), entry.getValue().getBytes()); + } + } + } catch (IOException e) { + throw new HiveException(e); } } diff --git ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java index 6497495..b0457be 100644 --- ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java +++ ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java @@ -1201,7 +1201,8 @@ public Vertex createVertex(JobConf conf, BaseWork work, /** * Set up credentials for the base work on secure clusters */ - public void addCredentials(BaseWork work, DAG dag) { + public void addCredentials(BaseWork work, DAG dag) throws IOException { + dag.setCredentials(UserGroupInformation.getCurrentUser().getCredentials()); if (work instanceof MapWork) { addCredentials((MapWork) work, dag); } else if (work instanceof ReduceWork) { diff --git ql/src/java/org/apache/hadoop/hive/ql/index/HiveIndexedInputFormat.java ql/src/java/org/apache/hadoop/hive/ql/index/HiveIndexedInputFormat.java index 0e6ec84..a02baf9 100644 --- ql/src/java/org/apache/hadoop/hive/ql/index/HiveIndexedInputFormat.java +++ ql/src/java/org/apache/hadoop/hive/ql/index/HiveIndexedInputFormat.java @@ -81,7 +81,12 @@ public HiveIndexedInputFormat(String indexFileName) { // class Class inputFormatClass = part.getInputFileFormatClass(); InputFormat inputFormat = getInputFormatFromCache(inputFormatClass, job); - Utilities.copyTableJobPropertiesToConf(part.getTableDesc(), newjob); + + try { + Utilities.copyTableJobPropertiesToConf(part.getTableDesc(), newjob); + } catch (HiveException e) { + throw new IOException(e); + } FileInputFormat.setInputPaths(newjob, dir); newjob.setInputFormat(inputFormat.getClass()); diff --git ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java index 010b88c..21394c6 100755 --- ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java +++ ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java @@ -357,9 +357,13 @@ public RecordReader getRecordReader(InputSplit split, JobConf job, LOG.debug("Found spec for " + hsplit.getPath() + " " + part + " from " + pathToPartitionInfo); } - if ((part != null) && (part.getTableDesc() != null)) { - Utilities.copyTableJobPropertiesToConf(part.getTableDesc(), job); - nonNative = part.getTableDesc().isNonNative(); + try { + if ((part != null) && (part.getTableDesc() != null)) { + Utilities.copyTableJobPropertiesToConf(part.getTableDesc(), job); + nonNative = part.getTableDesc().isNonNative(); + } + } catch (HiveException e) { + throw new IOException(e); } Path splitPath = hsplit.getPath(); @@ -419,7 +423,11 @@ private void addSplitsForGroup(List dirs, TableScanOperator tableScan, Job InputFormat inputFormat, Class inputFormatClass, int splits, TableDesc table, List result) throws IOException { - Utilities.copyTablePropertiesToConf(table, conf); + try { + Utilities.copyTablePropertiesToConf(table, conf); + } catch (HiveException e) { + throw new IOException(e); + } if (tableScan != null) { pushFilters(conf, tableScan); diff --git ql/src/java/org/apache/hadoop/hive/ql/io/parquet/ProjectionPusher.java ql/src/java/org/apache/hadoop/hive/ql/io/parquet/ProjectionPusher.java index 68407f5..42f9b66 100644 --- ql/src/java/org/apache/hadoop/hive/ql/io/parquet/ProjectionPusher.java +++ ql/src/java/org/apache/hadoop/hive/ql/io/parquet/ProjectionPusher.java @@ -183,9 +183,14 @@ public JobConf pushProjectionsAndFilters(JobConf jobConf, Path path) final JobConf cloneJobConf = new JobConf(jobConf); final PartitionDesc part = pathToPartitionInfo.get(path); - if ((part != null) && (part.getTableDesc() != null)) { - Utilities.copyTableJobPropertiesToConf(part.getTableDesc(), cloneJobConf); + try { + if ((part != null) && (part.getTableDesc() != null)) { + Utilities.copyTableJobPropertiesToConf(part.getTableDesc(), cloneJobConf); + } + } catch (Exception e) { + throw new IOException(e); } + pushProjectionsAndFilters(cloneJobConf, path.toString(), path.toUri().getPath()); return cloneJobConf; } diff --git ql/src/java/org/apache/hadoop/hive/ql/metadata/DefaultStorageHandler.java ql/src/java/org/apache/hadoop/hive/ql/metadata/DefaultStorageHandler.java index 82b78b8..e87a96d 100644 --- ql/src/java/org/apache/hadoop/hive/ql/metadata/DefaultStorageHandler.java +++ ql/src/java/org/apache/hadoop/hive/ql/metadata/DefaultStorageHandler.java @@ -93,6 +93,11 @@ public void configureJobConf(TableDesc tableDesc, JobConf jobConf) { } @Override + public void configureInputJobCredentials(TableDesc tableDesc, Map secrets) { + //do nothing by default + } + + @Override public Configuration getConf() { return conf; } diff --git ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java index 5975d0c..038f4a1 100644 --- ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java +++ ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveStorageHandler.java @@ -99,6 +99,12 @@ public abstract void configureInputJobProperties(TableDesc tableDesc, Map jobProperties); /** + * This method is called to allw the StorageHandlers the chance to + * populate secret keys into the job's credentials. + */ + public abstract void configureInputJobCredentials(TableDesc tableDesc, Map secrets); + + /** * 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). diff --git ql/src/java/org/apache/hadoop/hive/ql/parse/UnparseTranslator.java ql/src/java/org/apache/hadoop/hive/ql/parse/UnparseTranslator.java index 4ca8329..9a0a74d 100644 --- ql/src/java/org/apache/hadoop/hive/ql/parse/UnparseTranslator.java +++ ql/src/java/org/apache/hadoop/hive/ql/parse/UnparseTranslator.java @@ -116,7 +116,6 @@ void addTranslation(ASTNode node, String replacementText) { if (existingEntry.getValue().tokenStopIndex <= tokenStopIndex && existingEntry.getKey() >= tokenStartIndex) { // Collect newer entry is if a super-set of existing entry, - assert (replacementText.contains(existingEntry.getValue().replacementText)); subsetEntries.add(existingEntry.getKey()); // check if the existing entry contains the new } else if (existingEntry.getValue().tokenStopIndex >= tokenStopIndex && diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/PartitionDesc.java ql/src/java/org/apache/hadoop/hive/ql/plan/PartitionDesc.java index 68dcd0d..157a697 100644 --- ql/src/java/org/apache/hadoop/hive/ql/plan/PartitionDesc.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/PartitionDesc.java @@ -116,7 +116,11 @@ public PartitionDesc(final Partition part,final TableDesc tblDesc, private void PartitionDescConstructorHelper(final Partition part,final TableDesc tblDesc, boolean setInputFileFormat) throws HiveException { + + PlanUtils.configureInputJobPropertiesForStorageHandler(tblDesc); + this.tableDesc = tblDesc; + setPartSpec(part.getSpec()); if (setInputFileFormat) { setInputFileFormatClass(part.getInputFormatClass()); @@ -367,7 +371,6 @@ public int hashCode() { * URI to the partition file */ public void deriveBaseFileName(Path path) { - PlanUtils.configureInputJobPropertiesForStorageHandler(tableDesc); if (path == null) { return; diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java index 14f2a12..d82973c 100644 --- ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java @@ -896,6 +896,7 @@ private static void configureJobPropertiesForStorageHandler(boolean input, org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.META_TABLE_STORAGE)); if (storageHandler != null) { Map jobProperties = new LinkedHashMap(); + Map jobSecrets = new LinkedHashMap(); if(input) { try { storageHandler.configureInputJobProperties( @@ -906,6 +907,15 @@ private static void configureJobPropertiesForStorageHandler(boolean input, "using configureTableJobProperties",e); storageHandler.configureTableJobProperties(tableDesc, jobProperties); } + + try{ + storageHandler.configureInputJobCredentials( + tableDesc, + jobSecrets); + } catch(AbstractMethodError e) { + // ignore + LOG.info("configureInputJobSecrets not found"); + } } else { try { @@ -924,6 +934,11 @@ private static void configureJobPropertiesForStorageHandler(boolean input, if (!jobProperties.isEmpty()) { tableDesc.setJobProperties(jobProperties); } + + // same idea, only set for non-native tables + if (!jobSecrets.isEmpty()) { + tableDesc.setJobSecrets(jobSecrets); + } } } catch (HiveException ex) { throw new RuntimeException(ex); diff --git ql/src/java/org/apache/hadoop/hive/ql/plan/TableDesc.java ql/src/java/org/apache/hadoop/hive/ql/plan/TableDesc.java index 117aa14..8b7339d 100644 --- ql/src/java/org/apache/hadoop/hive/ql/plan/TableDesc.java +++ ql/src/java/org/apache/hadoop/hive/ql/plan/TableDesc.java @@ -53,6 +53,7 @@ private Class outputFileFormatClass; private java.util.Properties properties; private Map jobProperties; + private Map jobSecrets; public TableDesc() { } @@ -143,6 +144,14 @@ public void setJobProperties(Map jobProperties) { return jobProperties; } + public void setJobSecrets(Map jobSecrets) { + this.jobSecrets = jobSecrets; + } + + public Map getJobSecrets() { + return jobSecrets; + } + /** * @return the serdeClassName */ diff --git ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java index ed88725..18f77e0 100644 --- ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java +++ ql/src/java/org/apache/hadoop/hive/ql/ppd/OpProcFactory.java @@ -97,7 +97,7 @@ protected static final Logger LOG = LoggerFactory.getLogger(OpProcFactory.class .getName()); - private static ExprWalkerInfo getChildWalkerInfo(Operator current, OpWalkerInfo owi) { + private static ExprWalkerInfo getChildWalkerInfo(Operator current, OpWalkerInfo owi) throws SemanticException { if (current.getNumChild() == 0) { return null; } @@ -875,7 +875,7 @@ protected ExprWalkerInfo mergeChildrenPred(Node nd, OpWalkerInfo owi, } protected static Object createFilter(Operator op, - ExprWalkerInfo pushDownPreds, OpWalkerInfo owi) { + ExprWalkerInfo pushDownPreds, OpWalkerInfo owi) throws SemanticException { if (pushDownPreds != null && pushDownPreds.hasAnyCandidates()) { return createFilter(op, pushDownPreds.getFinalCandidates(), owi); } @@ -883,7 +883,7 @@ protected static Object createFilter(Operator op, } protected static Object createFilter(Operator op, - Map> predicates, OpWalkerInfo owi) { + Map> predicates, OpWalkerInfo owi) throws SemanticException { RowSchema inputRS = op.getSchema(); // combine all predicates into a single expression @@ -970,7 +970,7 @@ private static ExprNodeGenericFuncDesc pushFilterToStorageHandler( TableScanOperator tableScanOp, ExprNodeGenericFuncDesc originalPredicate, OpWalkerInfo owi, - HiveConf hiveConf) { + HiveConf hiveConf) throws SemanticException { TableScanDesc tableScanDesc = tableScanOp.getConf(); Table tbl = tableScanDesc.getTableMetadata(); @@ -997,9 +997,15 @@ private static ExprNodeGenericFuncDesc pushFilterToStorageHandler( JobConf jobConf = new JobConf(owi.getParseContext().getConf()); Utilities.setColumnNameList(jobConf, tableScanOp); Utilities.setColumnTypeList(jobConf, tableScanOp); - Utilities.copyTableJobPropertiesToConf( - Utilities.getTableDesc(tbl), - jobConf); + + try { + Utilities.copyTableJobPropertiesToConf( + Utilities.getTableDesc(tbl), + jobConf); + } catch (Exception e) { + throw new SemanticException(e); + } + Deserializer deserializer = tbl.getDeserializer(); HiveStoragePredicateHandler.DecomposedPredicate decomposed = predicateHandler.decomposePredicate( diff --git ql/src/test/org/apache/hadoop/hive/ql/exec/InputEstimatorTestClass.java ql/src/test/org/apache/hadoop/hive/ql/exec/InputEstimatorTestClass.java index 8c52979..f3f4388 100644 --- ql/src/test/org/apache/hadoop/hive/ql/exec/InputEstimatorTestClass.java +++ ql/src/test/org/apache/hadoop/hive/ql/exec/InputEstimatorTestClass.java @@ -75,6 +75,11 @@ public void configureInputJobProperties(TableDesc tableDesc, Map } @Override + public void configureInputJobCredentials(TableDesc tableDesc, Map jobProperties) { + + } + + @Override public void configureOutputJobProperties(TableDesc tableDesc, Map jobProperties) { } diff --git ql/src/test/queries/clientpositive/jdbc_handler.q ql/src/test/queries/clientpositive/jdbc_handler.q index a37e547..847f577 100644 --- ql/src/test/queries/clientpositive/jdbc_handler.q +++ ql/src/test/queries/clientpositive/jdbc_handler.q @@ -9,31 +9,24 @@ owner STRING ) STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' TBLPROPERTIES ( -"hive.sql.database.type" = "DERBY", -"hive.sql.jdbc.url" = "jdbc:derby:;databaseName=${test.tmp.dir}/junit_metastore_db;create=true", -"hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver", +"hive.sql.database.type" = "METASTORE", "hive.sql.query" = "SELECT TBL_ID, DB_ID, TBL_NAME, TBL_TYPE, OWNER FROM TBLS", -"hive.sql.column.mapping" = "id=TBL_ID, db_id=DB_ID, name=TBL_NAME, type=TBL_TYPE, owner=OWNER", -"hive.sql.dbcp.maxActive" = "1" +"hive.sql.column.mapping" = "id=TBL_ID, db_id=DB_ID, name=TBL_NAME, type=TBL_TYPE, owner=OWNER" ); CREATE EXTERNAL TABLE dbs ( -id int, -name STRING +DB_ID int, +NAME STRING ) STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' TBLPROPERTIES ( -"hive.sql.database.type" = "DERBY", -"hive.sql.jdbc.url" = "jdbc:derby:;databaseName=${test.tmp.dir}/junit_metastore_db;create=true", -"hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver", -"hive.sql.query" = "SELECT DB_ID, NAME FROM DBS", -"hive.sql.column.mapping" = "id=DB_ID, name=NAME", -"hive.sql.dbcp.maxActive" = "1" +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = "SELECT DB_ID, NAME FROM DBS" ); -select tables.name as tn, dbs.name as dn, tables.type as t -from tables join dbs on (tables.db_id = dbs.id) order by tn, dn, t; +select tables.name as tn, dbs.NAME as dn, tables.type as t +from tables join dbs on (tables.db_id = dbs.DB_ID) order by tn, dn, t; explain select diff --git ql/src/test/queries/clientpositive/sysdb.q ql/src/test/queries/clientpositive/sysdb.q new file mode 100644 index 0000000..d94a164 --- /dev/null +++ ql/src/test/queries/clientpositive/sysdb.q @@ -0,0 +1,117 @@ +set hive.strict.checks.cartesian.product=false; + +set hive.compute.query.using.stats=false; + +set hive.support.concurrency=true; + +set hive.cbo.enable=false; + +create table src_buck (key int, value string) clustered by(value) into 2 buckets; + +create table src_skew (key int) skewed by (key) on (1,2,3); + +CREATE TABLE scr_txn (key int, value string) + CLUSTERED BY (key) INTO 2 BUCKETS STORED AS ORC + TBLPROPERTIES ( + "transactional"="true", + "compactor.mapreduce.map.memory.mb"="2048", + "compactorthreshold.hive.compactor.delta.num.threshold"="4", + "compactorthreshold.hive.compactor.delta.pct.threshold"="0.5"); + +CREATE TEMPORARY TABLE src_tmp (key int, value string); + +CREATE TABLE moretypes (a decimal(10,2), b tinyint, c smallint, d int, e bigint, f varchar(10), g char(3)); + +show grant user hive_test_user; + +source ../../metastore/scripts/upgrade/hive/hive-schema-3.0.0.hive.sql; + +use sys; + +select bucket_col_name, integer_idx from bucketing_cols order by bucket_col_name, integer_idx limit 5; + +select count(*) from cds; + +select column_name, type_name, integer_idx from columns_v2 order by column_name, integer_idx limit 5; + +select param_key, param_value from database_params order by param_key, param_value limit 5; + +select db_location_uri, name, owner_name, owner_type from dbs order by name; + +select grantor, principal_name from db_privs order by grantor, principal_name limit 5; + +select grantor, principal_name from global_privs order by grantor, principal_name limit 5; + +select index_name, index_handler_class from idxs order by index_name limit 5; + +select param_key, param_value from index_params order by param_key, param_value limit 5; + +select part_name from partitions order by part_name limit 5; + +select pkey_name, pkey_type from partition_keys order by pkey_name limit 5; + +select part_key_val, integer_idx from partition_key_vals order by part_key_val, integer_idx limit 5; + +select param_key, param_value from partition_params order by param_key, param_value limit 5; + +select grantor, principal_name from part_col_privs order by grantor, principal_name limit 5; + +select grantor, principal_name from part_privs order by grantor, principal_name limit 5; + +select role_name from roles order by role_name limit 5; + +select principal_name, grantor from role_map order by principal_name, grantor limit 5; + +select count(*) from sds; + +select param_key, param_value from sd_params order by param_key, param_value limit 5; + +select sequence_name from sequence_table order by sequence_name limit 5; + +select name, slib from serdes order by name, slib limit 5; + +select param_key, param_value from serde_params order by param_key, param_value limit 5; + +select skewed_col_name from skewed_col_names order by skewed_col_name limit 5; + +select count(*) from skewed_col_value_loc_map; + +select count(*) from skewed_string_list; + +select count(*) from skewed_string_list_values; + +select count(*) from skewed_values; + +select column_name, `order` from sort_cols order by column_name limit 5; + +select param_key, param_value from table_params order by param_key, param_value limit 5; + +select tbl_name from tbls order by tbl_name limit 5; + +select column_name, grantor, principal_name from tbl_col_privs order by column_name, principal_name limit 5; + +select grantor, principal_name from tbl_privs order by grantor, principal_name limit 5; + +select table_name, column_name, num_nulls, num_distincts from tab_col_stats order by table_name, column_name limit 10; + +select table_name, partition_name, column_name, num_nulls, num_distincts from part_col_stats order by table_name, partition_name, column_name limit 10; + +select schema_version from version order by schema_version limit 5; + +select func_name, func_type from funcs order by func_name, func_type limit 5; + +select constraint_name from key_constraints order by constraint_name limit 5; + +use INFORMATION_SCHEMA; + +select count(*) from SCHEMATA; + +select * from TABLES order by TABLE_SCHEMA, TABLE_NAME; + +select * from TABLE_PRIVILEGES order by GRANTOR, GRANTEE, TABLE_SCHEMA, TABLE_NAME limit 10; + +select * from COLUMNS where TABLE_NAME = 'alltypesorc' or TABLE_NAME = 'moretypes' order by TABLE_SCHEMA, TABLE_NAME, ORDINAL_POSITION ; + +select * from COLUMN_PRIVILEGES order by GRANTOR, GRANTEE, TABLE_SCHEMA, TABLE_NAME, COLUMN_NAME limit 10; + +select TABLE_SCHEMA, TABLE_NAME from views order by TABLE_SCHEMA, TABLE_NAME; diff --git ql/src/test/results/clientpositive/llap/jdbc_handler.q.out ql/src/test/results/clientpositive/llap/jdbc_handler.q.out index 483b7f9..7c428e8 100644 --- ql/src/test/results/clientpositive/llap/jdbc_handler.q.out +++ ql/src/test/results/clientpositive/llap/jdbc_handler.q.out @@ -8,12 +8,9 @@ type STRING, ) STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' TBLPROPERTIES ( -"hive.sql.database.type" = "DERBY", -"hive.sql.jdbc.url" = "jdbc:derby:;databaseName=${test.tmp.dir}/junit_metastore_db;create=true", -"hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver", +"hive.sql.database.type" = "METASTORE", "hive.sql.query" = "SELECT TBL_ID, DB_ID, TBL_NAME, TBL_TYPE, OWNER FROM TBLS", #### A masked pattern was here #### -"hive.sql.dbcp.maxActive" = "1" ) PREHOOK: type: CREATETABLE PREHOOK: Output: database:default @@ -28,58 +25,47 @@ type STRING, ) STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' TBLPROPERTIES ( -"hive.sql.database.type" = "DERBY", -"hive.sql.jdbc.url" = "jdbc:derby:;databaseName=${test.tmp.dir}/junit_metastore_db;create=true", -"hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver", +"hive.sql.database.type" = "METASTORE", "hive.sql.query" = "SELECT TBL_ID, DB_ID, TBL_NAME, TBL_TYPE, OWNER FROM TBLS", #### A masked pattern was here #### -"hive.sql.dbcp.maxActive" = "1" ) POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@tables PREHOOK: query: CREATE EXTERNAL TABLE dbs ( -id int, -name STRING +DB_ID int, +NAME STRING ) STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' TBLPROPERTIES ( -"hive.sql.database.type" = "DERBY", -"hive.sql.jdbc.url" = "jdbc:derby:;databaseName=${test.tmp.dir}/junit_metastore_db;create=true", -"hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver", -"hive.sql.query" = "SELECT DB_ID, NAME FROM DBS", -"hive.sql.column.mapping" = "id=DB_ID, name=NAME", -"hive.sql.dbcp.maxActive" = "1" +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = "SELECT DB_ID, NAME FROM DBS" ) PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@dbs POSTHOOK: query: CREATE EXTERNAL TABLE dbs ( -id int, -name STRING +DB_ID int, +NAME STRING ) STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' TBLPROPERTIES ( -"hive.sql.database.type" = "DERBY", -"hive.sql.jdbc.url" = "jdbc:derby:;databaseName=${test.tmp.dir}/junit_metastore_db;create=true", -"hive.sql.jdbc.driver" = "org.apache.derby.jdbc.EmbeddedDriver", -"hive.sql.query" = "SELECT DB_ID, NAME FROM DBS", -"hive.sql.column.mapping" = "id=DB_ID, name=NAME", -"hive.sql.dbcp.maxActive" = "1" +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = "SELECT DB_ID, NAME FROM DBS" ) POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@dbs -PREHOOK: query: select tables.name as tn, dbs.name as dn, tables.type as t -from tables join dbs on (tables.db_id = dbs.id) order by tn, dn, t +PREHOOK: query: select tables.name as tn, dbs.NAME as dn, tables.type as t +from tables join dbs on (tables.db_id = dbs.DB_ID) order by tn, dn, t PREHOOK: type: QUERY PREHOOK: Input: default@dbs PREHOOK: Input: default@tables #### A masked pattern was here #### -POSTHOOK: query: select tables.name as tn, dbs.name as dn, tables.type as t -from tables join dbs on (tables.db_id = dbs.id) order by tn, dn, t +POSTHOOK: query: select tables.name as tn, dbs.NAME as dn, tables.type as t +from tables join dbs on (tables.db_id = dbs.DB_ID) order by tn, dn, t POSTHOOK: type: QUERY POSTHOOK: Input: default@dbs POSTHOOK: Input: default@tables diff --git ql/src/test/results/clientpositive/llap/sysdb.q.out ql/src/test/results/clientpositive/llap/sysdb.q.out new file mode 100644 index 0000000..0af28ee --- /dev/null +++ ql/src/test/results/clientpositive/llap/sysdb.q.out @@ -0,0 +1,3437 @@ +PREHOOK: query: create table src_buck (key int, value string) clustered by(value) into 2 buckets +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@src_buck +POSTHOOK: query: create table src_buck (key int, value string) clustered by(value) into 2 buckets +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@src_buck +PREHOOK: query: create table src_skew (key int) skewed by (key) on (1,2,3) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@src_skew +POSTHOOK: query: create table src_skew (key int) skewed by (key) on (1,2,3) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@src_skew +PREHOOK: query: CREATE TABLE scr_txn (key int, value string) + CLUSTERED BY (key) INTO 2 BUCKETS STORED AS ORC + TBLPROPERTIES ( + "transactional"="true", + "compactor.mapreduce.map.memory.mb"="2048", + "compactorthreshold.hive.compactor.delta.num.threshold"="4", + "compactorthreshold.hive.compactor.delta.pct.threshold"="0.5") +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@scr_txn +POSTHOOK: query: CREATE TABLE scr_txn (key int, value string) + CLUSTERED BY (key) INTO 2 BUCKETS STORED AS ORC + TBLPROPERTIES ( + "transactional"="true", + "compactor.mapreduce.map.memory.mb"="2048", + "compactorthreshold.hive.compactor.delta.num.threshold"="4", + "compactorthreshold.hive.compactor.delta.pct.threshold"="0.5") +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@scr_txn +PREHOOK: query: CREATE TEMPORARY TABLE src_tmp (key int, value string) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@src_tmp +POSTHOOK: query: CREATE TEMPORARY TABLE src_tmp (key int, value string) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@src_tmp +PREHOOK: query: CREATE TABLE moretypes (a decimal(10,2), b tinyint, c smallint, d int, e bigint, f varchar(10), g char(3)) +PREHOOK: type: CREATETABLE +PREHOOK: Output: database:default +PREHOOK: Output: default@moretypes +POSTHOOK: query: CREATE TABLE moretypes (a decimal(10,2), b tinyint, c smallint, d int, e bigint, f varchar(10), g char(3)) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: database:default +POSTHOOK: Output: default@moretypes +PREHOOK: query: show grant user hive_test_user +PREHOOK: type: SHOW_GRANT +POSTHOOK: query: show grant user hive_test_user +POSTHOOK: type: SHOW_GRANT +default alltypesorc hive_test_user USER DELETE true -1 hive_test_user +default alltypesorc hive_test_user USER INSERT true -1 hive_test_user +default alltypesorc hive_test_user USER SELECT true -1 hive_test_user +default alltypesorc hive_test_user USER UPDATE true -1 hive_test_user +default cbo_t1 hive_test_user USER DELETE true -1 hive_test_user +default cbo_t1 hive_test_user USER INSERT true -1 hive_test_user +default cbo_t1 hive_test_user USER SELECT true -1 hive_test_user +default cbo_t1 hive_test_user USER UPDATE true -1 hive_test_user +default cbo_t2 hive_test_user USER DELETE true -1 hive_test_user +default cbo_t2 hive_test_user USER INSERT true -1 hive_test_user +default cbo_t2 hive_test_user USER SELECT true -1 hive_test_user +default cbo_t2 hive_test_user USER UPDATE true -1 hive_test_user +default cbo_t3 hive_test_user USER DELETE true -1 hive_test_user +default cbo_t3 hive_test_user USER INSERT true -1 hive_test_user +default cbo_t3 hive_test_user USER SELECT true -1 hive_test_user +default cbo_t3 hive_test_user USER UPDATE true -1 hive_test_user +default lineitem hive_test_user USER DELETE true -1 hive_test_user +default lineitem hive_test_user USER INSERT true -1 hive_test_user +default lineitem hive_test_user USER SELECT true -1 hive_test_user +default lineitem hive_test_user USER UPDATE true -1 hive_test_user +default moretypes hive_test_user USER DELETE true -1 hive_test_user +default moretypes hive_test_user USER INSERT true -1 hive_test_user +default moretypes hive_test_user USER SELECT true -1 hive_test_user +default moretypes hive_test_user USER UPDATE true -1 hive_test_user +default part hive_test_user USER DELETE true -1 hive_test_user +default part hive_test_user USER INSERT true -1 hive_test_user +default part hive_test_user USER SELECT true -1 hive_test_user +default part hive_test_user USER UPDATE true -1 hive_test_user +default scr_txn hive_test_user USER DELETE true -1 hive_test_user +default scr_txn hive_test_user USER INSERT true -1 hive_test_user +default scr_txn hive_test_user USER SELECT true -1 hive_test_user +default scr_txn hive_test_user USER UPDATE true -1 hive_test_user +default src hive_test_user USER DELETE true -1 hive_test_user +default src hive_test_user USER INSERT true -1 hive_test_user +default src hive_test_user USER SELECT true -1 hive_test_user +default src hive_test_user USER UPDATE true -1 hive_test_user +default src1 hive_test_user USER DELETE true -1 hive_test_user +default src1 hive_test_user USER INSERT true -1 hive_test_user +default src1 hive_test_user USER SELECT true -1 hive_test_user +default src1 hive_test_user USER UPDATE true -1 hive_test_user +default src_buck hive_test_user USER DELETE true -1 hive_test_user +default src_buck hive_test_user USER INSERT true -1 hive_test_user +default src_buck hive_test_user USER SELECT true -1 hive_test_user +default src_buck hive_test_user USER UPDATE true -1 hive_test_user +default src_cbo hive_test_user USER DELETE true -1 hive_test_user +default src_cbo hive_test_user USER INSERT true -1 hive_test_user +default src_cbo hive_test_user USER SELECT true -1 hive_test_user +default src_cbo hive_test_user USER UPDATE true -1 hive_test_user +default src_json hive_test_user USER DELETE true -1 hive_test_user +default src_json hive_test_user USER INSERT true -1 hive_test_user +default src_json hive_test_user USER SELECT true -1 hive_test_user +default src_json hive_test_user USER UPDATE true -1 hive_test_user +default src_sequencefile hive_test_user USER DELETE true -1 hive_test_user +default src_sequencefile hive_test_user USER INSERT true -1 hive_test_user +default src_sequencefile hive_test_user USER SELECT true -1 hive_test_user +default src_sequencefile hive_test_user USER UPDATE true -1 hive_test_user +default src_skew hive_test_user USER DELETE true -1 hive_test_user +default src_skew hive_test_user USER INSERT true -1 hive_test_user +default src_skew hive_test_user USER SELECT true -1 hive_test_user +default src_skew hive_test_user USER UPDATE true -1 hive_test_user +default src_thrift hive_test_user USER DELETE true -1 hive_test_user +default src_thrift hive_test_user USER INSERT true -1 hive_test_user +default src_thrift hive_test_user USER SELECT true -1 hive_test_user +default src_thrift hive_test_user USER UPDATE true -1 hive_test_user +default srcbucket hive_test_user USER DELETE true -1 hive_test_user +default srcbucket hive_test_user USER INSERT true -1 hive_test_user +default srcbucket hive_test_user USER SELECT true -1 hive_test_user +default srcbucket hive_test_user USER UPDATE true -1 hive_test_user +default srcbucket2 hive_test_user USER DELETE true -1 hive_test_user +default srcbucket2 hive_test_user USER INSERT true -1 hive_test_user +default srcbucket2 hive_test_user USER SELECT true -1 hive_test_user +default srcbucket2 hive_test_user USER UPDATE true -1 hive_test_user +default srcpart hive_test_user USER DELETE true -1 hive_test_user +default srcpart hive_test_user USER INSERT true -1 hive_test_user +default srcpart hive_test_user USER SELECT true -1 hive_test_user +default srcpart hive_test_user USER UPDATE true -1 hive_test_user +PREHOOK: query: CREATE DATABASE SYS +PREHOOK: type: CREATEDATABASE +PREHOOK: Output: database:SYS +POSTHOOK: query: CREATE DATABASE SYS +POSTHOOK: type: CREATEDATABASE +POSTHOOK: Output: database:SYS +PREHOOK: query: USE SYS +PREHOOK: type: SWITCHDATABASE +PREHOOK: Input: database:sys +POSTHOOK: query: USE SYS +POSTHOOK: type: SWITCHDATABASE +POSTHOOK: Input: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `BUCKETING_COLS` ( + `SD_ID` bigint, + `BUCKET_COL_NAME` string, + `INTEGER_IDX` int, + CONSTRAINT `SYS_PK_BUCKETING_COLS` PRIMARY KEY (`SD_ID`,`INTEGER_IDX`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"SD_ID\", + \"BUCKET_COL_NAME\", + \"INTEGER_IDX\" +FROM + \"BUCKETING_COLS\"" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@BUCKETING_COLS +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `BUCKETING_COLS` ( + `SD_ID` bigint, + `BUCKET_COL_NAME` string, + `INTEGER_IDX` int, + CONSTRAINT `SYS_PK_BUCKETING_COLS` PRIMARY KEY (`SD_ID`,`INTEGER_IDX`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"SD_ID\", + \"BUCKET_COL_NAME\", + \"INTEGER_IDX\" +FROM + \"BUCKETING_COLS\"" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@BUCKETING_COLS +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `CDS` ( + `CD_ID` bigint, + CONSTRAINT `SYS_PK_CDS` PRIMARY KEY (`CD_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"CD_ID\" +FROM + \"CDS\"" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@CDS +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `CDS` ( + `CD_ID` bigint, + CONSTRAINT `SYS_PK_CDS` PRIMARY KEY (`CD_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"CD_ID\" +FROM + \"CDS\"" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@CDS +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `COLUMNS_V2` ( + `CD_ID` bigint, + `COMMENT` string, + `COLUMN_NAME` string, + `TYPE_NAME` string, + `INTEGER_IDX` int, + CONSTRAINT `SYS_PK_COLUMN_V2` PRIMARY KEY (`CD_ID`,`COLUMN_NAME`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"CD_ID\", + \"COMMENT\", + \"COLUMN_NAME\", + \"TYPE_NAME\", + \"INTEGER_IDX\" +FROM + \"COLUMNS_V2\"" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@COLUMNS_V2 +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `COLUMNS_V2` ( + `CD_ID` bigint, + `COMMENT` string, + `COLUMN_NAME` string, + `TYPE_NAME` string, + `INTEGER_IDX` int, + CONSTRAINT `SYS_PK_COLUMN_V2` PRIMARY KEY (`CD_ID`,`COLUMN_NAME`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"CD_ID\", + \"COMMENT\", + \"COLUMN_NAME\", + \"TYPE_NAME\", + \"INTEGER_IDX\" +FROM + \"COLUMNS_V2\"" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@COLUMNS_V2 +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `DATABASE_PARAMS` ( + `DB_ID` bigint, + `PARAM_KEY` string, + `PARAM_VALUE` string, + CONSTRAINT `SYS_PK_DATABASE_PARAMS` PRIMARY KEY (`DB_ID`,`PARAM_KEY`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"DB_ID\", + \"PARAM_KEY\", + \"PARAM_VALUE\" +FROM + \"DATABASE_PARAMS\"" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@DATABASE_PARAMS +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `DATABASE_PARAMS` ( + `DB_ID` bigint, + `PARAM_KEY` string, + `PARAM_VALUE` string, + CONSTRAINT `SYS_PK_DATABASE_PARAMS` PRIMARY KEY (`DB_ID`,`PARAM_KEY`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"DB_ID\", + \"PARAM_KEY\", + \"PARAM_VALUE\" +FROM + \"DATABASE_PARAMS\"" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@DATABASE_PARAMS +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `DBS` ( + `DB_ID` bigint, + `DB_LOCATION_URI` string, + `NAME` string, + `OWNER_NAME` string, + `OWNER_TYPE` string, + CONSTRAINT `SYS_PK_DBS` PRIMARY KEY (`DB_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"DB_ID\", + \"DB_LOCATION_URI\", + \"NAME\", + \"OWNER_NAME\", + \"OWNER_TYPE\" +FROM + DBS" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@DBS +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `DBS` ( + `DB_ID` bigint, + `DB_LOCATION_URI` string, + `NAME` string, + `OWNER_NAME` string, + `OWNER_TYPE` string, + CONSTRAINT `SYS_PK_DBS` PRIMARY KEY (`DB_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"DB_ID\", + \"DB_LOCATION_URI\", + \"NAME\", + \"OWNER_NAME\", + \"OWNER_TYPE\" +FROM + DBS" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@DBS +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `DB_PRIVS` ( + `DB_GRANT_ID` bigint, + `CREATE_TIME` int, + `DB_ID` bigint, + `GRANT_OPTION` int, + `GRANTOR` string, + `GRANTOR_TYPE` string, + `PRINCIPAL_NAME` string, + `PRINCIPAL_TYPE` string, + `DB_PRIV` string, + CONSTRAINT `SYS_PK_DB_PRIVS` PRIMARY KEY (`DB_GRANT_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"DB_GRANT_ID\", + \"CREATE_TIME\", + \"DB_ID\", + \"GRANT_OPTION\", + \"GRANTOR\", + \"GRANTOR_TYPE\", + \"PRINCIPAL_NAME\", + \"PRINCIPAL_TYPE\", + \"DB_PRIV\" +FROM + \"DB_PRIVS\"" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@DB_PRIVS +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `DB_PRIVS` ( + `DB_GRANT_ID` bigint, + `CREATE_TIME` int, + `DB_ID` bigint, + `GRANT_OPTION` int, + `GRANTOR` string, + `GRANTOR_TYPE` string, + `PRINCIPAL_NAME` string, + `PRINCIPAL_TYPE` string, + `DB_PRIV` string, + CONSTRAINT `SYS_PK_DB_PRIVS` PRIMARY KEY (`DB_GRANT_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"DB_GRANT_ID\", + \"CREATE_TIME\", + \"DB_ID\", + \"GRANT_OPTION\", + \"GRANTOR\", + \"GRANTOR_TYPE\", + \"PRINCIPAL_NAME\", + \"PRINCIPAL_TYPE\", + \"DB_PRIV\" +FROM + \"DB_PRIVS\"" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@DB_PRIVS +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `GLOBAL_PRIVS` ( + `USER_GRANT_ID` bigint, + `CREATE_TIME` int, + `GRANT_OPTION` string, + `GRANTOR` string, + `GRANTOR_TYPE` string, + `PRINCIPAL_NAME` string, + `PRINCIPAL_TYPE` string, + `USER_PRIV` string, + CONSTRAINT `SYS_PK_GLOBAL_PRIVS` PRIMARY KEY (`USER_GRANT_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"USER_GRANT_ID\", + \"CREATE_TIME\", + \"GRANT_OPTION\", + \"GRANTOR\", + \"GRANTOR_TYPE\", + \"PRINCIPAL_NAME\", + \"PRINCIPAL_TYPE\", + \"USER_PRIV\" +FROM + \"GLOBAL_PRIVS\"" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@GLOBAL_PRIVS +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `GLOBAL_PRIVS` ( + `USER_GRANT_ID` bigint, + `CREATE_TIME` int, + `GRANT_OPTION` string, + `GRANTOR` string, + `GRANTOR_TYPE` string, + `PRINCIPAL_NAME` string, + `PRINCIPAL_TYPE` string, + `USER_PRIV` string, + CONSTRAINT `SYS_PK_GLOBAL_PRIVS` PRIMARY KEY (`USER_GRANT_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"USER_GRANT_ID\", + \"CREATE_TIME\", + \"GRANT_OPTION\", + \"GRANTOR\", + \"GRANTOR_TYPE\", + \"PRINCIPAL_NAME\", + \"PRINCIPAL_TYPE\", + \"USER_PRIV\" +FROM + \"GLOBAL_PRIVS\"" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@GLOBAL_PRIVS +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `IDXS` ( + `INDEX_ID` bigint, + `CREATE_TIME` int, + `DEFERRED_REBUILD` boolean, + `INDEX_HANDLER_CLASS` string, + `INDEX_NAME` string, + `INDEX_TBL_ID` bigint, + `LAST_ACCESS_TIME` int, + `ORIG_TBL_ID` bigint, + `SD_ID` bigint, + CONSTRAINT `SYS_PK_IDXS` PRIMARY KEY (`INDEX_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"INDEX_ID\", + \"CREATE_TIME\", + \"DEFERRED_REBUILD\", + \"INDEX_HANDLER_CLASS\", + \"INDEX_NAME\", + \"INDEX_TBL_ID\", + \"LAST_ACCESS_TIME\", + \"ORIG_TBL_ID\", + \"SD_ID\" +FROM + \"IDXS\"" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@IDXS +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `IDXS` ( + `INDEX_ID` bigint, + `CREATE_TIME` int, + `DEFERRED_REBUILD` boolean, + `INDEX_HANDLER_CLASS` string, + `INDEX_NAME` string, + `INDEX_TBL_ID` bigint, + `LAST_ACCESS_TIME` int, + `ORIG_TBL_ID` bigint, + `SD_ID` bigint, + CONSTRAINT `SYS_PK_IDXS` PRIMARY KEY (`INDEX_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"INDEX_ID\", + \"CREATE_TIME\", + \"DEFERRED_REBUILD\", + \"INDEX_HANDLER_CLASS\", + \"INDEX_NAME\", + \"INDEX_TBL_ID\", + \"LAST_ACCESS_TIME\", + \"ORIG_TBL_ID\", + \"SD_ID\" +FROM + \"IDXS\"" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@IDXS +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `INDEX_PARAMS` ( + `INDEX_ID` bigint, + `PARAM_KEY` string, + `PARAM_VALUE` string, + CONSTRAINT `SYS_PK_INDEX_PARAMS` PRIMARY KEY (`INDEX_ID`,`PARAM_KEY`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"INDEX_ID\", + \"PARAM_KEY\", + \"PARAM_VALUE\" +FROM + \"INDEX_PARAMS\"" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@INDEX_PARAMS +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `INDEX_PARAMS` ( + `INDEX_ID` bigint, + `PARAM_KEY` string, + `PARAM_VALUE` string, + CONSTRAINT `SYS_PK_INDEX_PARAMS` PRIMARY KEY (`INDEX_ID`,`PARAM_KEY`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"INDEX_ID\", + \"PARAM_KEY\", + \"PARAM_VALUE\" +FROM + \"INDEX_PARAMS\"" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@INDEX_PARAMS +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `PARTITIONS` ( + `PART_ID` bigint, + `CREATE_TIME` int, + `LAST_ACCESS_TIME` int, + `PART_NAME` string, + `SD_ID` bigint, + `TBL_ID` bigint, + CONSTRAINT `SYS_PK_PARTITIONS` PRIMARY KEY (`PART_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"PART_ID\", + \"CREATE_TIME\", + \"LAST_ACCESS_TIME\", + \"PART_NAME\", + \"SD_ID\", + \"TBL_ID\" +FROM + \"PARTITIONS\"" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@PARTITIONS +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `PARTITIONS` ( + `PART_ID` bigint, + `CREATE_TIME` int, + `LAST_ACCESS_TIME` int, + `PART_NAME` string, + `SD_ID` bigint, + `TBL_ID` bigint, + CONSTRAINT `SYS_PK_PARTITIONS` PRIMARY KEY (`PART_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"PART_ID\", + \"CREATE_TIME\", + \"LAST_ACCESS_TIME\", + \"PART_NAME\", + \"SD_ID\", + \"TBL_ID\" +FROM + \"PARTITIONS\"" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@PARTITIONS +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `PARTITION_KEYS` ( + `TBL_ID` bigint, + `PKEY_COMMENT` string, + `PKEY_NAME` string, + `PKEY_TYPE` string, + `INTEGER_IDX` int, + CONSTRAINT `SYS_PK_PARTITION_KEYS` PRIMARY KEY (`TBL_ID`,`PKEY_NAME`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"TBL_ID\", + \"PKEY_COMMENT\", + \"PKEY_NAME\", + \"PKEY_TYPE\", + \"INTEGER_IDX\" +FROM + \"PARTITION_KEYS\"" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@PARTITION_KEYS +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `PARTITION_KEYS` ( + `TBL_ID` bigint, + `PKEY_COMMENT` string, + `PKEY_NAME` string, + `PKEY_TYPE` string, + `INTEGER_IDX` int, + CONSTRAINT `SYS_PK_PARTITION_KEYS` PRIMARY KEY (`TBL_ID`,`PKEY_NAME`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"TBL_ID\", + \"PKEY_COMMENT\", + \"PKEY_NAME\", + \"PKEY_TYPE\", + \"INTEGER_IDX\" +FROM + \"PARTITION_KEYS\"" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@PARTITION_KEYS +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `PARTITION_KEY_VALS` ( + `PART_ID` bigint, + `PART_KEY_VAL` string, + `INTEGER_IDX` int, + CONSTRAINT `SYS_PK_PARTITION_KEY_VALS` PRIMARY KEY (`PART_ID`,`INTEGER_IDX`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"PART_ID\", + \"PART_KEY_VAL\", + \"INTEGER_IDX\" +FROM + \"PARTITION_KEY_VALS\"" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@PARTITION_KEY_VALS +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `PARTITION_KEY_VALS` ( + `PART_ID` bigint, + `PART_KEY_VAL` string, + `INTEGER_IDX` int, + CONSTRAINT `SYS_PK_PARTITION_KEY_VALS` PRIMARY KEY (`PART_ID`,`INTEGER_IDX`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"PART_ID\", + \"PART_KEY_VAL\", + \"INTEGER_IDX\" +FROM + \"PARTITION_KEY_VALS\"" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@PARTITION_KEY_VALS +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `PARTITION_PARAMS` ( + `PART_ID` bigint, + `PARAM_KEY` string, + `PARAM_VALUE` string, + CONSTRAINT `SYS_PK_PARTITION_PARAMS` PRIMARY KEY (`PART_ID`,`PARAM_KEY`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"PART_ID\", + \"PARAM_KEY\", + \"PARAM_VALUE\" +FROM + \"PARTITION_PARAMS\"" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@PARTITION_PARAMS +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `PARTITION_PARAMS` ( + `PART_ID` bigint, + `PARAM_KEY` string, + `PARAM_VALUE` string, + CONSTRAINT `SYS_PK_PARTITION_PARAMS` PRIMARY KEY (`PART_ID`,`PARAM_KEY`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"PART_ID\", + \"PARAM_KEY\", + \"PARAM_VALUE\" +FROM + \"PARTITION_PARAMS\"" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@PARTITION_PARAMS +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `PART_COL_PRIVS` ( + `PART_COLUMN_GRANT_ID` bigint, + `COLUMN_NAME` string, + `CREATE_TIME` int, + `GRANT_OPTION` int, + `GRANTOR` string, + `GRANTOR_TYPE` string, + `PART_ID` bigint, + `PRINCIPAL_NAME` string, + `PRINCIPAL_TYPE` string, + `PART_COL_PRIV` string, + CONSTRAINT `SYS_PK_PART_COL_PRIVS` PRIMARY KEY (`PART_COLUMN_GRANT_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"PART_COLUMN_GRANT_ID\", + \"COLUMN_NAME\", + \"CREATE_TIME\", + \"GRANT_OPTION\", + \"GRANTOR\", + \"GRANTOR_TYPE\", + \"PART_ID\", + \"PRINCIPAL_NAME\", + \"PRINCIPAL_TYPE\", + \"PART_COL_PRIV\" +FROM + \"PART_COL_PRIVS\"" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@PART_COL_PRIVS +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `PART_COL_PRIVS` ( + `PART_COLUMN_GRANT_ID` bigint, + `COLUMN_NAME` string, + `CREATE_TIME` int, + `GRANT_OPTION` int, + `GRANTOR` string, + `GRANTOR_TYPE` string, + `PART_ID` bigint, + `PRINCIPAL_NAME` string, + `PRINCIPAL_TYPE` string, + `PART_COL_PRIV` string, + CONSTRAINT `SYS_PK_PART_COL_PRIVS` PRIMARY KEY (`PART_COLUMN_GRANT_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"PART_COLUMN_GRANT_ID\", + \"COLUMN_NAME\", + \"CREATE_TIME\", + \"GRANT_OPTION\", + \"GRANTOR\", + \"GRANTOR_TYPE\", + \"PART_ID\", + \"PRINCIPAL_NAME\", + \"PRINCIPAL_TYPE\", + \"PART_COL_PRIV\" +FROM + \"PART_COL_PRIVS\"" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@PART_COL_PRIVS +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `PART_PRIVS` ( + `PART_GRANT_ID` bigint, + `CREATE_TIME` int, + `GRANT_OPTION` int, + `GRANTOR` string, + `GRANTOR_TYPE` string, + `PART_ID` bigint, + `PRINCIPAL_NAME` string, + `PRINCIPAL_TYPE` string, + `PART_PRIV` string, + CONSTRAINT `SYS_PK_PART_PRIVS` PRIMARY KEY (`PART_GRANT_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"PART_GRANT_ID\", + \"CREATE_TIME\", + \"GRANT_OPTION\", + \"GRANTOR\", + \"GRANTOR_TYPE\", + \"PART_ID\", + \"PRINCIPAL_NAME\", + \"PRINCIPAL_TYPE\", + \"PART_PRIV\" +FROM + \"PART_PRIVS\"" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@PART_PRIVS +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `PART_PRIVS` ( + `PART_GRANT_ID` bigint, + `CREATE_TIME` int, + `GRANT_OPTION` int, + `GRANTOR` string, + `GRANTOR_TYPE` string, + `PART_ID` bigint, + `PRINCIPAL_NAME` string, + `PRINCIPAL_TYPE` string, + `PART_PRIV` string, + CONSTRAINT `SYS_PK_PART_PRIVS` PRIMARY KEY (`PART_GRANT_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"PART_GRANT_ID\", + \"CREATE_TIME\", + \"GRANT_OPTION\", + \"GRANTOR\", + \"GRANTOR_TYPE\", + \"PART_ID\", + \"PRINCIPAL_NAME\", + \"PRINCIPAL_TYPE\", + \"PART_PRIV\" +FROM + \"PART_PRIVS\"" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@PART_PRIVS +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `ROLES` ( + `ROLE_ID` bigint, + `CREATE_TIME` int, + `OWNER_NAME` string, + `ROLE_NAME` string, + CONSTRAINT `SYS_PK_ROLES` PRIMARY KEY (`ROLE_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"ROLE_ID\", + \"CREATE_TIME\", + \"OWNER_NAME\", + \"ROLE_NAME\" +FROM + \"ROLES\"" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@ROLES +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `ROLES` ( + `ROLE_ID` bigint, + `CREATE_TIME` int, + `OWNER_NAME` string, + `ROLE_NAME` string, + CONSTRAINT `SYS_PK_ROLES` PRIMARY KEY (`ROLE_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"ROLE_ID\", + \"CREATE_TIME\", + \"OWNER_NAME\", + \"ROLE_NAME\" +FROM + \"ROLES\"" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@ROLES +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `ROLE_MAP` ( + `ROLE_GRANT_ID` bigint, + `ADD_TIME` int, + `GRANT_OPTION` int, + `GRANTOR` string, + `GRANTOR_TYPE` string, + `PRINCIPAL_NAME` string, + `PRINCIPAL_TYPE` string, + `ROLE_ID` bigint, + CONSTRAINT `SYS_PK_ROLE_MAP` PRIMARY KEY (`ROLE_GRANT_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"ROLE_GRANT_ID\", + \"ADD_TIME\", + \"GRANT_OPTION\", + \"GRANTOR\", + \"GRANTOR_TYPE\", + \"PRINCIPAL_NAME\", + \"PRINCIPAL_TYPE\", + \"ROLE_ID\" +FROM + \"ROLE_MAP\"" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@ROLE_MAP +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `ROLE_MAP` ( + `ROLE_GRANT_ID` bigint, + `ADD_TIME` int, + `GRANT_OPTION` int, + `GRANTOR` string, + `GRANTOR_TYPE` string, + `PRINCIPAL_NAME` string, + `PRINCIPAL_TYPE` string, + `ROLE_ID` bigint, + CONSTRAINT `SYS_PK_ROLE_MAP` PRIMARY KEY (`ROLE_GRANT_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"ROLE_GRANT_ID\", + \"ADD_TIME\", + \"GRANT_OPTION\", + \"GRANTOR\", + \"GRANTOR_TYPE\", + \"PRINCIPAL_NAME\", + \"PRINCIPAL_TYPE\", + \"ROLE_ID\" +FROM + \"ROLE_MAP\"" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@ROLE_MAP +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `SDS` ( + `SD_ID` bigint, + `CD_ID` bigint, + `INPUT_FORMAT` string, + `IS_COMPRESSED` boolean, + `IS_STOREDASSUBDIRECTORIES` boolean, + `LOCATION` string, + `NUM_BUCKETS` int, + `OUTPUT_FORMAT` string, + `SERDE_ID` bigint, + CONSTRAINT `SYS_PK_SDS` PRIMARY KEY (`SD_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"SD_ID\", + \"CD_ID\", + \"INPUT_FORMAT\", + \"IS_COMPRESSED\", + \"IS_STOREDASSUBDIRECTORIES\", + \"LOCATION\", + \"NUM_BUCKETS\", + \"OUTPUT_FORMAT\", + \"SERDE_ID\" +FROM + \"SDS\"" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@SDS +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `SDS` ( + `SD_ID` bigint, + `CD_ID` bigint, + `INPUT_FORMAT` string, + `IS_COMPRESSED` boolean, + `IS_STOREDASSUBDIRECTORIES` boolean, + `LOCATION` string, + `NUM_BUCKETS` int, + `OUTPUT_FORMAT` string, + `SERDE_ID` bigint, + CONSTRAINT `SYS_PK_SDS` PRIMARY KEY (`SD_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"SD_ID\", + \"CD_ID\", + \"INPUT_FORMAT\", + \"IS_COMPRESSED\", + \"IS_STOREDASSUBDIRECTORIES\", + \"LOCATION\", + \"NUM_BUCKETS\", + \"OUTPUT_FORMAT\", + \"SERDE_ID\" +FROM + \"SDS\"" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@SDS +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `SD_PARAMS` ( + `SD_ID` bigint, + `PARAM_KEY` string, + `PARAM_VALUE` string, + CONSTRAINT `SYS_PK_SD_PARAMS` PRIMARY KEY (`SD_ID`,`PARAM_KEY`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"SD_ID\", + \"PARAM_KEY\", + \"PARAM_VALUE\" +FROM + \"SD_PARAMS\"" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@SD_PARAMS +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `SD_PARAMS` ( + `SD_ID` bigint, + `PARAM_KEY` string, + `PARAM_VALUE` string, + CONSTRAINT `SYS_PK_SD_PARAMS` PRIMARY KEY (`SD_ID`,`PARAM_KEY`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"SD_ID\", + \"PARAM_KEY\", + \"PARAM_VALUE\" +FROM + \"SD_PARAMS\"" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@SD_PARAMS +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `SEQUENCE_TABLE` ( + `SEQUENCE_NAME` string, + `NEXT_VAL` bigint, + CONSTRAINT `SYS_PK_SEQUENCE_TABLE` PRIMARY KEY (`SEQUENCE_NAME`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"SEQUENCE_NAME\", + \"NEXT_VAL\" +FROM + \"SEQUENCE_TABLE\"" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@SEQUENCE_TABLE +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `SEQUENCE_TABLE` ( + `SEQUENCE_NAME` string, + `NEXT_VAL` bigint, + CONSTRAINT `SYS_PK_SEQUENCE_TABLE` PRIMARY KEY (`SEQUENCE_NAME`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"SEQUENCE_NAME\", + \"NEXT_VAL\" +FROM + \"SEQUENCE_TABLE\"" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@SEQUENCE_TABLE +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `SERDES` ( + `SERDE_ID` bigint, + `NAME` string, + `SLIB` string, + CONSTRAINT `SYS_PK_SERDES` PRIMARY KEY (`SERDE_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"SERDE_ID\", + \"NAME\", + \"SLIB\" +FROM + \"SERDES\"" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@SERDES +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `SERDES` ( + `SERDE_ID` bigint, + `NAME` string, + `SLIB` string, + CONSTRAINT `SYS_PK_SERDES` PRIMARY KEY (`SERDE_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"SERDE_ID\", + \"NAME\", + \"SLIB\" +FROM + \"SERDES\"" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@SERDES +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `SERDE_PARAMS` ( + `SERDE_ID` bigint, + `PARAM_KEY` string, + `PARAM_VALUE` string, + CONSTRAINT `SYS_PK_SERDE_PARAMS` PRIMARY KEY (`SERDE_ID`,`PARAM_KEY`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"SERDE_ID\", + \"PARAM_KEY\", + \"PARAM_VALUE\" +FROM + \"SERDE_PARAMS\"" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@SERDE_PARAMS +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `SERDE_PARAMS` ( + `SERDE_ID` bigint, + `PARAM_KEY` string, + `PARAM_VALUE` string, + CONSTRAINT `SYS_PK_SERDE_PARAMS` PRIMARY KEY (`SERDE_ID`,`PARAM_KEY`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"SERDE_ID\", + \"PARAM_KEY\", + \"PARAM_VALUE\" +FROM + \"SERDE_PARAMS\"" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@SERDE_PARAMS +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `SKEWED_COL_NAMES` ( + `SD_ID` bigint, + `SKEWED_COL_NAME` string, + `INTEGER_IDX` int, + CONSTRAINT `SYS_PK_SKEWED_COL_NAMES` PRIMARY KEY (`SD_ID`,`INTEGER_IDX`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"SD_ID\", + \"SKEWED_COL_NAME\", + \"INTEGER_IDX\" +FROM + \"SKEWED_COL_NAMES\"" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@SKEWED_COL_NAMES +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `SKEWED_COL_NAMES` ( + `SD_ID` bigint, + `SKEWED_COL_NAME` string, + `INTEGER_IDX` int, + CONSTRAINT `SYS_PK_SKEWED_COL_NAMES` PRIMARY KEY (`SD_ID`,`INTEGER_IDX`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"SD_ID\", + \"SKEWED_COL_NAME\", + \"INTEGER_IDX\" +FROM + \"SKEWED_COL_NAMES\"" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@SKEWED_COL_NAMES +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `SKEWED_COL_VALUE_LOC_MAP` ( + `SD_ID` bigint, + `STRING_LIST_ID_KID` bigint, + `LOCATION` string, + CONSTRAINT `SYS_PK_COL_VALUE_LOC_MAP` PRIMARY KEY (`SD_ID`,`STRING_LIST_ID_KID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"SD_ID\", + \"STRING_LIST_ID_KID\", + \"LOCATION\" +FROM + \"SKEWED_COL_VALUE_LOC_MAP\"" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@SKEWED_COL_VALUE_LOC_MAP +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `SKEWED_COL_VALUE_LOC_MAP` ( + `SD_ID` bigint, + `STRING_LIST_ID_KID` bigint, + `LOCATION` string, + CONSTRAINT `SYS_PK_COL_VALUE_LOC_MAP` PRIMARY KEY (`SD_ID`,`STRING_LIST_ID_KID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"SD_ID\", + \"STRING_LIST_ID_KID\", + \"LOCATION\" +FROM + \"SKEWED_COL_VALUE_LOC_MAP\"" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@SKEWED_COL_VALUE_LOC_MAP +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `SKEWED_STRING_LIST` ( + `STRING_LIST_ID` bigint, + CONSTRAINT `SYS_PK_SKEWED_STRING_LIST` PRIMARY KEY (`STRING_LIST_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"STRING_LIST_ID\" +FROM + \"SKEWED_STRING_LIST\"" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@SKEWED_STRING_LIST +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `SKEWED_STRING_LIST` ( + `STRING_LIST_ID` bigint, + CONSTRAINT `SYS_PK_SKEWED_STRING_LIST` PRIMARY KEY (`STRING_LIST_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"STRING_LIST_ID\" +FROM + \"SKEWED_STRING_LIST\"" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@SKEWED_STRING_LIST +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `SKEWED_STRING_LIST_VALUES` ( + `STRING_LIST_ID` bigint, + `STRING_LIST_VALUE` string, + `INTEGER_IDX` int, + CONSTRAINT `SYS_PK_SKEWED_STRING_LIST_VALUES` PRIMARY KEY (`STRING_LIST_ID`,`INTEGER_IDX`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"STRING_LIST_ID\", + \"STRING_LIST_VALUE\", + \"INTEGER_IDX\" +FROM + \"SKEWED_STRING_LIST_VALUES\"" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@SKEWED_STRING_LIST_VALUES +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `SKEWED_STRING_LIST_VALUES` ( + `STRING_LIST_ID` bigint, + `STRING_LIST_VALUE` string, + `INTEGER_IDX` int, + CONSTRAINT `SYS_PK_SKEWED_STRING_LIST_VALUES` PRIMARY KEY (`STRING_LIST_ID`,`INTEGER_IDX`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"STRING_LIST_ID\", + \"STRING_LIST_VALUE\", + \"INTEGER_IDX\" +FROM + \"SKEWED_STRING_LIST_VALUES\"" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@SKEWED_STRING_LIST_VALUES +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `SKEWED_VALUES` ( + `SD_ID_OID` bigint, + `STRING_LIST_ID_EID` bigint, + `INTEGER_IDX` int, + CONSTRAINT `SYS_PK_SKEWED_VALUES` PRIMARY KEY (`SD_ID_OID`,`INTEGER_IDX`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"SD_ID_OID\", + \"STRING_LIST_ID_EID\", + \"INTEGER_IDX\" +FROM + \"SKEWED_VALUES\"" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@SKEWED_VALUES +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `SKEWED_VALUES` ( + `SD_ID_OID` bigint, + `STRING_LIST_ID_EID` bigint, + `INTEGER_IDX` int, + CONSTRAINT `SYS_PK_SKEWED_VALUES` PRIMARY KEY (`SD_ID_OID`,`INTEGER_IDX`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"SD_ID_OID\", + \"STRING_LIST_ID_EID\", + \"INTEGER_IDX\" +FROM + \"SKEWED_VALUES\"" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@SKEWED_VALUES +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `SORT_COLS` ( + `SD_ID` bigint, + `COLUMN_NAME` string, + `ORDER` int, + `INTEGER_IDX` int, + CONSTRAINT `SYS_PK_SORT_COLS` PRIMARY KEY (`SD_ID`,`INTEGER_IDX`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"SD_ID\", + \"COLUMN_NAME\", + \"ORDER\", + \"INTEGER_IDX\" +FROM + \"SORT_COLS\"" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@SORT_COLS +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `SORT_COLS` ( + `SD_ID` bigint, + `COLUMN_NAME` string, + `ORDER` int, + `INTEGER_IDX` int, + CONSTRAINT `SYS_PK_SORT_COLS` PRIMARY KEY (`SD_ID`,`INTEGER_IDX`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"SD_ID\", + \"COLUMN_NAME\", + \"ORDER\", + \"INTEGER_IDX\" +FROM + \"SORT_COLS\"" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@SORT_COLS +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `TABLE_PARAMS` ( + `TBL_ID` bigint, + `PARAM_KEY` string, + `PARAM_VALUE` string, + CONSTRAINT `SYS_PK_TABLE_PARAMS` PRIMARY KEY (`TBL_ID`,`PARAM_KEY`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"TBL_ID\", + \"PARAM_KEY\", + \"PARAM_VALUE\" +FROM + \"TABLE_PARAMS\"" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@TABLE_PARAMS +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `TABLE_PARAMS` ( + `TBL_ID` bigint, + `PARAM_KEY` string, + `PARAM_VALUE` string, + CONSTRAINT `SYS_PK_TABLE_PARAMS` PRIMARY KEY (`TBL_ID`,`PARAM_KEY`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"TBL_ID\", + \"PARAM_KEY\", + \"PARAM_VALUE\" +FROM + \"TABLE_PARAMS\"" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@TABLE_PARAMS +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `TBLS` ( + `TBL_ID` bigint, + `CREATE_TIME` int, + `DB_ID` bigint, + `LAST_ACCESS_TIME` int, + `OWNER` string, + `RETENTION` int, + `SD_ID` bigint, + `TBL_NAME` string, + `TBL_TYPE` string, + `VIEW_EXPANDED_TEXT` string, + `VIEW_ORIGINAL_TEXT` string, + `IS_REWRITE_ENABLED` boolean, + CONSTRAINT `SYS_PK_TBLS` PRIMARY KEY (`TBL_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"TBL_ID\", + \"CREATE_TIME\", + \"DB_ID\", + \"LAST_ACCESS_TIME\", + \"OWNER\", + \"RETENTION\", + \"SD_ID\", + \"TBL_NAME\", + \"TBL_TYPE\", + \"VIEW_EXPANDED_TEXT\", + \"VIEW_ORIGINAL_TEXT\", + \"IS_REWRITE_ENABLED\" +FROM TBLS" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@TBLS +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `TBLS` ( + `TBL_ID` bigint, + `CREATE_TIME` int, + `DB_ID` bigint, + `LAST_ACCESS_TIME` int, + `OWNER` string, + `RETENTION` int, + `SD_ID` bigint, + `TBL_NAME` string, + `TBL_TYPE` string, + `VIEW_EXPANDED_TEXT` string, + `VIEW_ORIGINAL_TEXT` string, + `IS_REWRITE_ENABLED` boolean, + CONSTRAINT `SYS_PK_TBLS` PRIMARY KEY (`TBL_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"TBL_ID\", + \"CREATE_TIME\", + \"DB_ID\", + \"LAST_ACCESS_TIME\", + \"OWNER\", + \"RETENTION\", + \"SD_ID\", + \"TBL_NAME\", + \"TBL_TYPE\", + \"VIEW_EXPANDED_TEXT\", + \"VIEW_ORIGINAL_TEXT\", + \"IS_REWRITE_ENABLED\" +FROM TBLS" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@TBLS +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `TBL_COL_PRIVS` ( + `TBL_COLUMN_GRANT_ID` bigint, + `COLUMN_NAME` string, + `CREATE_TIME` int, + `GRANT_OPTION` int, + `GRANTOR` string, + `GRANTOR_TYPE` string, + `PRINCIPAL_NAME` string, + `PRINCIPAL_TYPE` string, + `TBL_COL_PRIV` string, + `TBL_ID` bigint, + CONSTRAINT `SYS_PK_TBL_COL_PRIVS` PRIMARY KEY (`TBL_COLUMN_GRANT_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"TBL_COLUMN_GRANT_ID\", + \"COLUMN_NAME\", + \"CREATE_TIME\", + \"GRANT_OPTION\", + \"GRANTOR\", + \"GRANTOR_TYPE\", + \"PRINCIPAL_NAME\", + \"PRINCIPAL_TYPE\", + \"TBL_COL_PRIV\", + \"TBL_ID\" +FROM + \"TBL_COL_PRIVS\"" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@TBL_COL_PRIVS +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `TBL_COL_PRIVS` ( + `TBL_COLUMN_GRANT_ID` bigint, + `COLUMN_NAME` string, + `CREATE_TIME` int, + `GRANT_OPTION` int, + `GRANTOR` string, + `GRANTOR_TYPE` string, + `PRINCIPAL_NAME` string, + `PRINCIPAL_TYPE` string, + `TBL_COL_PRIV` string, + `TBL_ID` bigint, + CONSTRAINT `SYS_PK_TBL_COL_PRIVS` PRIMARY KEY (`TBL_COLUMN_GRANT_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"TBL_COLUMN_GRANT_ID\", + \"COLUMN_NAME\", + \"CREATE_TIME\", + \"GRANT_OPTION\", + \"GRANTOR\", + \"GRANTOR_TYPE\", + \"PRINCIPAL_NAME\", + \"PRINCIPAL_TYPE\", + \"TBL_COL_PRIV\", + \"TBL_ID\" +FROM + \"TBL_COL_PRIVS\"" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@TBL_COL_PRIVS +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `TBL_PRIVS` ( + `TBL_GRANT_ID` bigint, + `CREATE_TIME` int, + `GRANT_OPTION` int, + `GRANTOR` string, + `GRANTOR_TYPE` string, + `PRINCIPAL_NAME` string, + `PRINCIPAL_TYPE` string, + `TBL_PRIV` string, + `TBL_ID` bigint, + CONSTRAINT `SYS_PK_TBL_PRIVS` PRIMARY KEY (`TBL_GRANT_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"TBL_GRANT_ID\", + \"CREATE_TIME\", + \"GRANT_OPTION\", + \"GRANTOR\", + \"GRANTOR_TYPE\", + \"PRINCIPAL_NAME\", + \"PRINCIPAL_TYPE\", + \"TBL_PRIV\", + \"TBL_ID\" +FROM + \"TBL_PRIVS\"" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@TBL_PRIVS +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `TBL_PRIVS` ( + `TBL_GRANT_ID` bigint, + `CREATE_TIME` int, + `GRANT_OPTION` int, + `GRANTOR` string, + `GRANTOR_TYPE` string, + `PRINCIPAL_NAME` string, + `PRINCIPAL_TYPE` string, + `TBL_PRIV` string, + `TBL_ID` bigint, + CONSTRAINT `SYS_PK_TBL_PRIVS` PRIMARY KEY (`TBL_GRANT_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"TBL_GRANT_ID\", + \"CREATE_TIME\", + \"GRANT_OPTION\", + \"GRANTOR\", + \"GRANTOR_TYPE\", + \"PRINCIPAL_NAME\", + \"PRINCIPAL_TYPE\", + \"TBL_PRIV\", + \"TBL_ID\" +FROM + \"TBL_PRIVS\"" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@TBL_PRIVS +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `TAB_COL_STATS` ( + `CS_ID` bigint, + `DB_NAME` string, + `TABLE_NAME` string, + `COLUMN_NAME` string, + `COLUMN_TYPE` string, + `TBL_ID` bigint, + `LONG_LOW_VALUE` bigint, + `LONG_HIGH_VALUE` bigint, + `DOUBLE_HIGH_VALUE` double, + `DOUBLE_LOW_VALUE` double, + `BIG_DECIMAL_LOW_VALUE` string, + `BIG_DECIMAL_HIGH_VALUE` string, + `NUM_NULLS` bigint, + `NUM_DISTINCTS` bigint, + `AVG_COL_LEN` double, + `MAX_COL_LEN` bigint, + `NUM_TRUES` bigint, + `NUM_FALSES` bigint, + `LAST_ANALYZED` bigint, + CONSTRAINT `SYS_PK_TAB_COL_STATS` PRIMARY KEY (`CS_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"CS_ID\", + \"DB_NAME\", + \"TABLE_NAME\", + \"COLUMN_NAME\", + \"COLUMN_TYPE\", + \"TBL_ID\", + \"LONG_LOW_VALUE\", + \"LONG_HIGH_VALUE\", + \"DOUBLE_HIGH_VALUE\", + \"DOUBLE_LOW_VALUE\", + \"BIG_DECIMAL_LOW_VALUE\", + \"BIG_DECIMAL_HIGH_VALUE\", + \"NUM_NULLS\", + \"NUM_DISTINCTS\", + \"AVG_COL_LEN\", + \"MAX_COL_LEN\", + \"NUM_TRUES\", + \"NUM_FALSES\", + \"LAST_ANALYZED\" +FROM + \"TAB_COL_STATS\"" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@TAB_COL_STATS +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `TAB_COL_STATS` ( + `CS_ID` bigint, + `DB_NAME` string, + `TABLE_NAME` string, + `COLUMN_NAME` string, + `COLUMN_TYPE` string, + `TBL_ID` bigint, + `LONG_LOW_VALUE` bigint, + `LONG_HIGH_VALUE` bigint, + `DOUBLE_HIGH_VALUE` double, + `DOUBLE_LOW_VALUE` double, + `BIG_DECIMAL_LOW_VALUE` string, + `BIG_DECIMAL_HIGH_VALUE` string, + `NUM_NULLS` bigint, + `NUM_DISTINCTS` bigint, + `AVG_COL_LEN` double, + `MAX_COL_LEN` bigint, + `NUM_TRUES` bigint, + `NUM_FALSES` bigint, + `LAST_ANALYZED` bigint, + CONSTRAINT `SYS_PK_TAB_COL_STATS` PRIMARY KEY (`CS_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"CS_ID\", + \"DB_NAME\", + \"TABLE_NAME\", + \"COLUMN_NAME\", + \"COLUMN_TYPE\", + \"TBL_ID\", + \"LONG_LOW_VALUE\", + \"LONG_HIGH_VALUE\", + \"DOUBLE_HIGH_VALUE\", + \"DOUBLE_LOW_VALUE\", + \"BIG_DECIMAL_LOW_VALUE\", + \"BIG_DECIMAL_HIGH_VALUE\", + \"NUM_NULLS\", + \"NUM_DISTINCTS\", + \"AVG_COL_LEN\", + \"MAX_COL_LEN\", + \"NUM_TRUES\", + \"NUM_FALSES\", + \"LAST_ANALYZED\" +FROM + \"TAB_COL_STATS\"" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@TAB_COL_STATS +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `PART_COL_STATS` ( + `CS_ID` bigint, + `DB_NAME` string, + `TABLE_NAME` string, + `PARTITION_NAME` string, + `COLUMN_NAME` string, + `COLUMN_TYPE` string, + `PART_ID` bigint, + `LONG_LOW_VALUE` bigint, + `LONG_HIGH_VALUE` bigint, + `DOUBLE_HIGH_VALUE` double, + `DOUBLE_LOW_VALUE` double, + `BIG_DECIMAL_LOW_VALUE` string, + `BIG_DECIMAL_HIGH_VALUE` string, + `NUM_NULLS` bigint, + `NUM_DISTINCTS` bigint, + `AVG_COL_LEN` double, + `MAX_COL_LEN` bigint, + `NUM_TRUES` bigint, + `NUM_FALSES` bigint, + `LAST_ANALYZED` bigint, + CONSTRAINT `SYS_PK_PART_COL_STATS` PRIMARY KEY (`CS_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"CS_ID\", + \"DB_NAME\", + \"TABLE_NAME\", + \"PARTITION_NAME\", + \"COLUMN_NAME\", + \"COLUMN_TYPE\", + \"PART_ID\", + \"LONG_LOW_VALUE\", + \"LONG_HIGH_VALUE\", + \"DOUBLE_HIGH_VALUE\", + \"DOUBLE_LOW_VALUE\", + \"BIG_DECIMAL_LOW_VALUE\", + \"BIG_DECIMAL_HIGH_VALUE\", + \"NUM_NULLS\", + \"NUM_DISTINCTS\", + \"AVG_COL_LEN\", + \"MAX_COL_LEN\", + \"NUM_TRUES\", + \"NUM_FALSES\", + \"LAST_ANALYZED\" +FROM + \"PART_COL_STATS\"" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@PART_COL_STATS +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `PART_COL_STATS` ( + `CS_ID` bigint, + `DB_NAME` string, + `TABLE_NAME` string, + `PARTITION_NAME` string, + `COLUMN_NAME` string, + `COLUMN_TYPE` string, + `PART_ID` bigint, + `LONG_LOW_VALUE` bigint, + `LONG_HIGH_VALUE` bigint, + `DOUBLE_HIGH_VALUE` double, + `DOUBLE_LOW_VALUE` double, + `BIG_DECIMAL_LOW_VALUE` string, + `BIG_DECIMAL_HIGH_VALUE` string, + `NUM_NULLS` bigint, + `NUM_DISTINCTS` bigint, + `AVG_COL_LEN` double, + `MAX_COL_LEN` bigint, + `NUM_TRUES` bigint, + `NUM_FALSES` bigint, + `LAST_ANALYZED` bigint, + CONSTRAINT `SYS_PK_PART_COL_STATS` PRIMARY KEY (`CS_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"CS_ID\", + \"DB_NAME\", + \"TABLE_NAME\", + \"PARTITION_NAME\", + \"COLUMN_NAME\", + \"COLUMN_TYPE\", + \"PART_ID\", + \"LONG_LOW_VALUE\", + \"LONG_HIGH_VALUE\", + \"DOUBLE_HIGH_VALUE\", + \"DOUBLE_LOW_VALUE\", + \"BIG_DECIMAL_LOW_VALUE\", + \"BIG_DECIMAL_HIGH_VALUE\", + \"NUM_NULLS\", + \"NUM_DISTINCTS\", + \"AVG_COL_LEN\", + \"MAX_COL_LEN\", + \"NUM_TRUES\", + \"NUM_FALSES\", + \"LAST_ANALYZED\" +FROM + \"PART_COL_STATS\"" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@PART_COL_STATS +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `VERSION` ( + `VER_ID` BIGINT, + `SCHEMA_VERSION` string, + `VERSION_COMMENT` string, + CONSTRAINT `SYS_PK_VERSION` PRIMARY KEY (`VER_ID`) DISABLE NOVALIDATE +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@VERSION +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `VERSION` ( + `VER_ID` BIGINT, + `SCHEMA_VERSION` string, + `VERSION_COMMENT` string, + CONSTRAINT `SYS_PK_VERSION` PRIMARY KEY (`VER_ID`) DISABLE NOVALIDATE +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@VERSION +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `DB_VERSION` ( + `VER_ID` BIGINT, + `SCHEMA_VERSION` string, + `VERSION_COMMENT` string, + CONSTRAINT `SYS_PK_DB_VERSION` PRIMARY KEY (`VER_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"VER_ID\", + \"SCHEMA_VERSION\", + \"VERSION_COMMENT\" +FROM + \"VERSION\"" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@DB_VERSION +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `DB_VERSION` ( + `VER_ID` BIGINT, + `SCHEMA_VERSION` string, + `VERSION_COMMENT` string, + CONSTRAINT `SYS_PK_DB_VERSION` PRIMARY KEY (`VER_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"VER_ID\", + \"SCHEMA_VERSION\", + \"VERSION_COMMENT\" +FROM + \"VERSION\"" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@DB_VERSION +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `FUNCS` ( + `FUNC_ID` bigint, + `CLASS_NAME` string, + `CREATE_TIME` int, + `DB_ID` bigint, + `FUNC_NAME` string, + `FUNC_TYPE` int, + `OWNER_NAME` string, + `OWNER_TYPE` string, + CONSTRAINT `SYS_PK_FUNCS` PRIMARY KEY (`FUNC_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"FUNC_ID\", + \"CLASS_NAME\", + \"CREATE_TIME\", + \"DB_ID\", + \"FUNC_NAME\", + \"FUNC_TYPE\", + \"OWNER_NAME\", + \"OWNER_TYPE\" +FROM + \"FUNCS\"" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@FUNCS +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `FUNCS` ( + `FUNC_ID` bigint, + `CLASS_NAME` string, + `CREATE_TIME` int, + `DB_ID` bigint, + `FUNC_NAME` string, + `FUNC_TYPE` int, + `OWNER_NAME` string, + `OWNER_TYPE` string, + CONSTRAINT `SYS_PK_FUNCS` PRIMARY KEY (`FUNC_ID`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"FUNC_ID\", + \"CLASS_NAME\", + \"CREATE_TIME\", + \"DB_ID\", + \"FUNC_NAME\", + \"FUNC_TYPE\", + \"OWNER_NAME\", + \"OWNER_TYPE\" +FROM + \"FUNCS\"" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@FUNCS +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE TABLE IF NOT EXISTS `KEY_CONSTRAINTS` +( + `CHILD_CD_ID` bigint, + `CHILD_INTEGER_IDX` int, + `CHILD_TBL_ID` bigint, + `PARENT_CD_ID` bigint, + `PARENT_INTEGER_IDX` int, + `PARENT_TBL_ID` bigint, + `POSITION` bigint, + `CONSTRAINT_NAME` string, + `CONSTRAINT_TYPE` string, + `UPDATE_RULE` string, + `DELETE_RULE` string, + `ENABLE_VALIDATE_RELY` int, + CONSTRAINT `SYS_PK_KEY_CONSTRAINTS` PRIMARY KEY (`CONSTRAINT_NAME`, `POSITION`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"CHILD_CD_ID\", + \"CHILD_INTEGER_IDX\", + \"CHILD_TBL_ID\", + \"PARENT_CD_ID\", + \"PARENT_INTEGER_IDX\", + \"PARENT_TBL_ID\", + \"POSITION\", + \"CONSTRAINT_NAME\", + \"CONSTRAINT_TYPE\", + \"UPDATE_RULE\", + \"DELETE_RULE\", + \"ENABLE_VALIDATE_RELY\" +FROM + \"KEY_CONSTRAINTS\"" +) +PREHOOK: type: CREATETABLE +PREHOOK: Output: SYS@KEY_CONSTRAINTS +PREHOOK: Output: database:sys +POSTHOOK: query: CREATE TABLE IF NOT EXISTS `KEY_CONSTRAINTS` +( + `CHILD_CD_ID` bigint, + `CHILD_INTEGER_IDX` int, + `CHILD_TBL_ID` bigint, + `PARENT_CD_ID` bigint, + `PARENT_INTEGER_IDX` int, + `PARENT_TBL_ID` bigint, + `POSITION` bigint, + `CONSTRAINT_NAME` string, + `CONSTRAINT_TYPE` string, + `UPDATE_RULE` string, + `DELETE_RULE` string, + `ENABLE_VALIDATE_RELY` int, + CONSTRAINT `SYS_PK_KEY_CONSTRAINTS` PRIMARY KEY (`CONSTRAINT_NAME`, `POSITION`) DISABLE NOVALIDATE +) +STORED BY 'org.apache.hive.storage.jdbc.JdbcStorageHandler' +TBLPROPERTIES ( +"hive.sql.database.type" = "METASTORE", +"hive.sql.query" = +"SELECT + \"CHILD_CD_ID\", + \"CHILD_INTEGER_IDX\", + \"CHILD_TBL_ID\", + \"PARENT_CD_ID\", + \"PARENT_INTEGER_IDX\", + \"PARENT_TBL_ID\", + \"POSITION\", + \"CONSTRAINT_NAME\", + \"CONSTRAINT_TYPE\", + \"UPDATE_RULE\", + \"DELETE_RULE\", + \"ENABLE_VALIDATE_RELY\" +FROM + \"KEY_CONSTRAINTS\"" +) +POSTHOOK: type: CREATETABLE +POSTHOOK: Output: SYS@KEY_CONSTRAINTS +POSTHOOK: Output: database:sys +PREHOOK: query: CREATE DATABASE INFORMATION_SCHEMA +PREHOOK: type: CREATEDATABASE +PREHOOK: Output: database:INFORMATION_SCHEMA +POSTHOOK: query: CREATE DATABASE INFORMATION_SCHEMA +POSTHOOK: type: CREATEDATABASE +POSTHOOK: Output: database:INFORMATION_SCHEMA +PREHOOK: query: USE INFORMATION_SCHEMA +PREHOOK: type: SWITCHDATABASE +PREHOOK: Input: database:information_schema +POSTHOOK: query: USE INFORMATION_SCHEMA +POSTHOOK: type: SWITCHDATABASE +POSTHOOK: Input: database:information_schema +PREHOOK: query: CREATE VIEW IF NOT EXISTS `SCHEMATA` +( + `CATALOG_NAME`, + `SCHEMA_NAME`, + `SCHEMA_OWNER`, + `DEFAULT_CHARACTER_SET_CATALOG`, + `DEFAULT_CHARACTER_SET_SCHEMA`, + `DEFAULT_CHARACTER_SET_NAME`, + `SQL_PATH` +) AS +SELECT + 'default', + `NAME`, + `OWNER_NAME`, + cast(null as string), + cast(null as string), + cast(null as string), + `DB_LOCATION_URI` +FROM + sys.DBS +PREHOOK: type: CREATEVIEW +PREHOOK: Input: sys@dbs +PREHOOK: Output: INFORMATION_SCHEMA@SCHEMATA +PREHOOK: Output: database:information_schema +POSTHOOK: query: CREATE VIEW IF NOT EXISTS `SCHEMATA` +( + `CATALOG_NAME`, + `SCHEMA_NAME`, + `SCHEMA_OWNER`, + `DEFAULT_CHARACTER_SET_CATALOG`, + `DEFAULT_CHARACTER_SET_SCHEMA`, + `DEFAULT_CHARACTER_SET_NAME`, + `SQL_PATH` +) AS +SELECT + 'default', + `NAME`, + `OWNER_NAME`, + cast(null as string), + cast(null as string), + cast(null as string), + `DB_LOCATION_URI` +FROM + sys.DBS +POSTHOOK: type: CREATEVIEW +POSTHOOK: Input: sys@dbs +POSTHOOK: Output: INFORMATION_SCHEMA@SCHEMATA +POSTHOOK: Output: database:information_schema +POSTHOOK: Lineage: SCHEMATA.catalog_name SIMPLE [] +POSTHOOK: Lineage: SCHEMATA.default_character_set_catalog EXPRESSION [] +POSTHOOK: Lineage: SCHEMATA.default_character_set_name EXPRESSION [] +POSTHOOK: Lineage: SCHEMATA.default_character_set_schema EXPRESSION [] +POSTHOOK: Lineage: SCHEMATA.schema_name SIMPLE [(dbs)dbs.FieldSchema(name:name, type:string, comment:from deserializer), ] +#### A masked pattern was here #### +POSTHOOK: Lineage: SCHEMATA.sql_path SIMPLE [(dbs)dbs.FieldSchema(name:db_location_uri, type:string, comment:from deserializer), ] +PREHOOK: query: CREATE VIEW IF NOT EXISTS `TABLES` +( + `TABLE_CATALOG`, + `TABLE_SCHEMA`, + `TABLE_NAME`, + `TABLE_TYPE`, + `SELF_REFERENCING_COLUMN_NAME`, + `REFERENCE_GENERATION`, + `USER_DEFINED_TYPE_CATALOG`, + `USER_DEFINED_TYPE_SCHEMA`, + `USER_DEFINED_TYPE_NAME`, + `IS_INSERTABLE_INTO`, + `IS_TYPED`, + `COMMIT_ACTION` +) AS +SELECT + 'default', + D.NAME, + T.TBL_NAME, + IF(length(T.VIEW_ORIGINAL_TEXT) > 0, 'VIEW', 'BASE_TABLE'), + cast(null as string), + cast(null as string), + cast(null as string), + cast(null as string), + cast(null as string), + IF(length(T.VIEW_ORIGINAL_TEXT) > 0, 'NO', 'YES'), + 'NO', + cast(null as string) +FROM + `sys`.`TBLS` T, `sys`.`DBS` D +WHERE + D.`DB_ID` = T.`DB_ID` +PREHOOK: type: CREATEVIEW +PREHOOK: Input: sys@dbs +PREHOOK: Input: sys@tbls +PREHOOK: Output: INFORMATION_SCHEMA@TABLES +PREHOOK: Output: database:information_schema +POSTHOOK: query: CREATE VIEW IF NOT EXISTS `TABLES` +( + `TABLE_CATALOG`, + `TABLE_SCHEMA`, + `TABLE_NAME`, + `TABLE_TYPE`, + `SELF_REFERENCING_COLUMN_NAME`, + `REFERENCE_GENERATION`, + `USER_DEFINED_TYPE_CATALOG`, + `USER_DEFINED_TYPE_SCHEMA`, + `USER_DEFINED_TYPE_NAME`, + `IS_INSERTABLE_INTO`, + `IS_TYPED`, + `COMMIT_ACTION` +) AS +SELECT + 'default', + D.NAME, + T.TBL_NAME, + IF(length(T.VIEW_ORIGINAL_TEXT) > 0, 'VIEW', 'BASE_TABLE'), + cast(null as string), + cast(null as string), + cast(null as string), + cast(null as string), + cast(null as string), + IF(length(T.VIEW_ORIGINAL_TEXT) > 0, 'NO', 'YES'), + 'NO', + cast(null as string) +FROM + `sys`.`TBLS` T, `sys`.`DBS` D +WHERE + D.`DB_ID` = T.`DB_ID` +POSTHOOK: type: CREATEVIEW +POSTHOOK: Input: sys@dbs +POSTHOOK: Input: sys@tbls +POSTHOOK: Output: INFORMATION_SCHEMA@TABLES +POSTHOOK: Output: database:information_schema +POSTHOOK: Lineage: TABLES.commit_action EXPRESSION [] +POSTHOOK: Lineage: TABLES.is_insertable_into EXPRESSION [(tbls)t.FieldSchema(name:view_original_text, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: TABLES.is_typed SIMPLE [] +POSTHOOK: Lineage: TABLES.reference_generation EXPRESSION [] +POSTHOOK: Lineage: TABLES.self_referencing_column_name EXPRESSION [] +POSTHOOK: Lineage: TABLES.table_catalog SIMPLE [] +POSTHOOK: Lineage: TABLES.table_name SIMPLE [(tbls)t.FieldSchema(name:tbl_name, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: TABLES.table_schema SIMPLE [(dbs)d.FieldSchema(name:name, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: TABLES.table_type EXPRESSION [(tbls)t.FieldSchema(name:view_original_text, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: TABLES.user_defined_type_catalog EXPRESSION [] +POSTHOOK: Lineage: TABLES.user_defined_type_name EXPRESSION [] +POSTHOOK: Lineage: TABLES.user_defined_type_schema EXPRESSION [] +PREHOOK: query: CREATE VIEW IF NOT EXISTS `TABLE_PRIVILEGES` +( + `GRANTOR`, + `GRANTEE`, + `TABLE_CATALOG`, + `TABLE_SCHEMA`, + `TABLE_NAME`, + `PRIVILEGE_TYPE`, + `IS_GRANTABLE`, + `WITH_HIERARCHY` +) AS +SELECT + `GRANTOR`, + `PRINCIPAL_NAME`, + 'default', + D.`NAME`, + T.`TBL_NAME`, + P.`TBL_PRIV`, + IF (P.`GRANT_OPTION` == 0, 'NO', 'YES'), + 'NO' +FROM + sys.`TBL_PRIVS` P, + sys.`TBLS` T, + sys.`DBS` D +WHERE + P.TBL_ID = T.TBL_ID + AND T.DB_ID = D.DB_ID +PREHOOK: type: CREATEVIEW +PREHOOK: Input: sys@dbs +PREHOOK: Input: sys@tbl_privs +PREHOOK: Input: sys@tbls +PREHOOK: Output: INFORMATION_SCHEMA@TABLE_PRIVILEGES +PREHOOK: Output: database:information_schema +POSTHOOK: query: CREATE VIEW IF NOT EXISTS `TABLE_PRIVILEGES` +( + `GRANTOR`, + `GRANTEE`, + `TABLE_CATALOG`, + `TABLE_SCHEMA`, + `TABLE_NAME`, + `PRIVILEGE_TYPE`, + `IS_GRANTABLE`, + `WITH_HIERARCHY` +) AS +SELECT + `GRANTOR`, + `PRINCIPAL_NAME`, + 'default', + D.`NAME`, + T.`TBL_NAME`, + P.`TBL_PRIV`, + IF (P.`GRANT_OPTION` == 0, 'NO', 'YES'), + 'NO' +FROM + sys.`TBL_PRIVS` P, + sys.`TBLS` T, + sys.`DBS` D +WHERE + P.TBL_ID = T.TBL_ID + AND T.DB_ID = D.DB_ID +POSTHOOK: type: CREATEVIEW +POSTHOOK: Input: sys@dbs +POSTHOOK: Input: sys@tbl_privs +POSTHOOK: Input: sys@tbls +POSTHOOK: Output: INFORMATION_SCHEMA@TABLE_PRIVILEGES +POSTHOOK: Output: database:information_schema +POSTHOOK: Lineage: TABLE_PRIVILEGES.grantee SIMPLE [(tbl_privs)p.FieldSchema(name:principal_name, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: TABLE_PRIVILEGES.grantor SIMPLE [(tbl_privs)p.FieldSchema(name:grantor, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: TABLE_PRIVILEGES.is_grantable EXPRESSION [(tbl_privs)p.FieldSchema(name:grant_option, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: TABLE_PRIVILEGES.privilege_type SIMPLE [(tbl_privs)p.FieldSchema(name:tbl_priv, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: TABLE_PRIVILEGES.table_catalog SIMPLE [] +POSTHOOK: Lineage: TABLE_PRIVILEGES.table_name SIMPLE [(tbls)t.FieldSchema(name:tbl_name, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: TABLE_PRIVILEGES.table_schema SIMPLE [(dbs)d.FieldSchema(name:name, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: TABLE_PRIVILEGES.with_hierarchy SIMPLE [] +PREHOOK: query: CREATE VIEW IF NOT EXISTS `COLUMNS` +( + `TABLE_CATALOG`, + `TABLE_SCHEMA`, + `TABLE_NAME`, + `COLUMN_NAME`, + `ORDINAL_POSITION`, + `COLUMN_DEFAULT`, + `IS_NULLABLE`, + `DATA_TYPE`, + `CHARACTER_MAXIMUM_LENGTH`, + `CHARACTER_OCTET_LENGTH`, + `NUMERIC_PRECISION`, + `NUMERIC_PRECISION_RADIX`, + `NUMERIC_SCALE`, + `DATETIME_PRECISION`, + `INTERVAL_TYPE`, + `INTERVAL_PRECISION`, + `CHARACTER_SET_CATALOG`, + `CHARACTER_SET_SCHEMA`, + `CHARACTER_SET_NAME`, + `COLLATION_CATALOG`, + `COLLATION_SCHEMA`, + `COLLATION_NAME`, + `UDT_CATALOG`, + `UDT_SCHEMA`, + `UDT_NAME`, + `SCOPE_CATALOG`, + `SCOPE_SCHEMA`, + `SCOPE_NAME`, + `MAXIMUM_CARDINALITY`, + `DTD_IDENTIFIER`, + `IS_SELF_REFERENCING`, + `IS_IDENTITY`, + `IDENTITY_GENERATION`, + `IDENTITY_START`, + `IDENTITY_INCREMENT`, + `IDENTITY_MAXIMUM`, + `IDENTITY_MINIMUM`, + `IDENTITY_CYCLE`, + `IS_GENERATED`, + `GENERATION_EXPRESSION`, + `IS_SYSTEM_TIME_PERIOD_START`, + `IS_SYSTEM_TIME_PERIOD_END`, + `SYSTEM_TIME_PERIOD_TIMESTAMP_GENERATION`, + `IS_UPDATABLE`, + `DECLARED_DATA_TYPE`, + `DECLARED_NUMERIC_PRECISION`, + `DECLARED_NUMERIC_SCALE` +) AS +SELECT + 'default', + D.NAME, + T.TBL_NAME, + C.COLUMN_NAME, + C.INTEGER_IDX, + cast (null as string), + 'YES', + C.TYPE_NAME as TYPE_NAME, + CASE WHEN lower(C.TYPE_NAME) like 'varchar%' THEN cast(regexp_extract(upper(C.TYPE_NAME), '^VARCHAR\\s*\\((\\d+)\\s*\\)$', 1) as int) + WHEN lower(C.TYPE_NAME) like 'char%' THEN cast(regexp_extract(upper(C.TYPE_NAME), '^CHAR\\s*\\((\\d+)\\s*\\)$', 1) as int) + ELSE null END, + CASE WHEN lower(C.TYPE_NAME) like 'varchar%' THEN cast(regexp_extract(upper(C.TYPE_NAME), '^VARCHAR\\s*\\((\\d+)\\s*\\)$', 1) as int) + WHEN lower(C.TYPE_NAME) like 'char%' THEN cast(regexp_extract(upper(C.TYPE_NAME), '^CHAR\\s*\\((\\d+)\\s*\\)$', 1) as int) + ELSE null END, + CASE WHEN lower(C.TYPE_NAME) = 'bigint' THEN 19 + WHEN lower(C.TYPE_NAME) = 'int' THEN 10 + WHEN lower(C.TYPE_NAME) = 'smallint' THEN 5 + WHEN lower(C.TYPE_NAME) = 'tinyint' THEN 3 + WHEN lower(C.TYPE_NAME) = 'float' THEN 23 + WHEN lower(C.TYPE_NAME) = 'double' THEN 53 + WHEN lower(C.TYPE_NAME) like 'decimal%' THEN regexp_extract(upper(C.TYPE_NAME), '^DECIMAL\\s*\\((\\d+)',1) + WHEN lower(C.TYPE_NAME) like 'numeric%' THEN regexp_extract(upper(C.TYPE_NAME), '^NUMERIC\\s*\\((\\d+)',1) + ELSE null END, + CASE WHEN lower(C.TYPE_NAME) = 'bigint' THEN 10 + WHEN lower(C.TYPE_NAME) = 'int' THEN 10 + WHEN lower(C.TYPE_NAME) = 'smallint' THEN 10 + WHEN lower(C.TYPE_NAME) = 'tinyint' THEN 10 + WHEN lower(C.TYPE_NAME) = 'float' THEN 2 + WHEN lower(C.TYPE_NAME) = 'double' THEN 2 + WHEN lower(C.TYPE_NAME) like 'decimal%' THEN 10 + WHEN lower(C.TYPE_NAME) like 'numeric%' THEN 10 + ELSE null END, + CASE WHEN lower(C.TYPE_NAME) like 'decimal%' THEN regexp_extract(upper(C.TYPE_NAME), '^DECIMAL\\s*\\((\\d+),(\\d+)',2) + WHEN lower(C.TYPE_NAME) like 'numeric%' THEN regexp_extract(upper(C.TYPE_NAME), '^NUMERIC\\s*\\((\\d+),(\\d+)',2) + ELSE null END, + CASE WHEN lower(C.TYPE_NAME) = 'date' THEN 0 + WHEN lower(C.TYPE_NAME) = 'timestamp' THEN 9 + ELSE null END, + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + C.CD_ID, + 'NO', + 'NO', + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + 'NEVER', + cast (null as string), + 'NO', + 'NO', + cast (null as string), + 'YES', + C.TYPE_NAME as DECLARED_DATA_TYPE, + CASE WHEN lower(C.TYPE_NAME) = 'bigint' THEN 19 + WHEN lower(C.TYPE_NAME) = 'int' THEN 10 + WHEN lower(C.TYPE_NAME) = 'smallint' THEN 5 + WHEN lower(C.TYPE_NAME) = 'tinyint' THEN 3 + WHEN lower(C.TYPE_NAME) = 'float' THEN 23 + WHEN lower(C.TYPE_NAME) = 'double' THEN 53 + WHEN lower(C.TYPE_NAME) like 'decimal%' THEN regexp_extract(upper(C.TYPE_NAME), '^DECIMAL\\s*\\((\\d+)',1) + WHEN lower(C.TYPE_NAME) like 'numeric%' THEN regexp_extract(upper(C.TYPE_NAME), '^NUMERIC\\s*\\((\\d+)',1) + ELSE null END, + CASE WHEN lower(C.TYPE_NAME) = 'bigint' THEN 10 + WHEN lower(C.TYPE_NAME) = 'int' THEN 10 + WHEN lower(C.TYPE_NAME) = 'smallint' THEN 10 + WHEN lower(C.TYPE_NAME) = 'tinyint' THEN 10 + WHEN lower(C.TYPE_NAME) = 'float' THEN 2 + WHEN lower(C.TYPE_NAME) = 'double' THEN 2 + WHEN lower(C.TYPE_NAME) like 'decimal%' THEN 10 + WHEN lower(C.TYPE_NAME) like 'numeric%' THEN 10 + ELSE null END +FROM + sys.`COLUMNS_V2` C, + sys.`SDS` S, + sys.`TBLS` T, + sys.`DBS` D +WHERE + S.`SD_ID` = T.`SD_ID` + AND T.`DB_ID` = D.`DB_ID` + AND C.`CD_ID` = S.`CD_ID` +PREHOOK: type: CREATEVIEW +PREHOOK: Input: sys@columns_v2 +PREHOOK: Input: sys@dbs +PREHOOK: Input: sys@sds +PREHOOK: Input: sys@tbls +PREHOOK: Output: INFORMATION_SCHEMA@COLUMNS +PREHOOK: Output: database:information_schema +POSTHOOK: query: CREATE VIEW IF NOT EXISTS `COLUMNS` +( + `TABLE_CATALOG`, + `TABLE_SCHEMA`, + `TABLE_NAME`, + `COLUMN_NAME`, + `ORDINAL_POSITION`, + `COLUMN_DEFAULT`, + `IS_NULLABLE`, + `DATA_TYPE`, + `CHARACTER_MAXIMUM_LENGTH`, + `CHARACTER_OCTET_LENGTH`, + `NUMERIC_PRECISION`, + `NUMERIC_PRECISION_RADIX`, + `NUMERIC_SCALE`, + `DATETIME_PRECISION`, + `INTERVAL_TYPE`, + `INTERVAL_PRECISION`, + `CHARACTER_SET_CATALOG`, + `CHARACTER_SET_SCHEMA`, + `CHARACTER_SET_NAME`, + `COLLATION_CATALOG`, + `COLLATION_SCHEMA`, + `COLLATION_NAME`, + `UDT_CATALOG`, + `UDT_SCHEMA`, + `UDT_NAME`, + `SCOPE_CATALOG`, + `SCOPE_SCHEMA`, + `SCOPE_NAME`, + `MAXIMUM_CARDINALITY`, + `DTD_IDENTIFIER`, + `IS_SELF_REFERENCING`, + `IS_IDENTITY`, + `IDENTITY_GENERATION`, + `IDENTITY_START`, + `IDENTITY_INCREMENT`, + `IDENTITY_MAXIMUM`, + `IDENTITY_MINIMUM`, + `IDENTITY_CYCLE`, + `IS_GENERATED`, + `GENERATION_EXPRESSION`, + `IS_SYSTEM_TIME_PERIOD_START`, + `IS_SYSTEM_TIME_PERIOD_END`, + `SYSTEM_TIME_PERIOD_TIMESTAMP_GENERATION`, + `IS_UPDATABLE`, + `DECLARED_DATA_TYPE`, + `DECLARED_NUMERIC_PRECISION`, + `DECLARED_NUMERIC_SCALE` +) AS +SELECT + 'default', + D.NAME, + T.TBL_NAME, + C.COLUMN_NAME, + C.INTEGER_IDX, + cast (null as string), + 'YES', + C.TYPE_NAME as TYPE_NAME, + CASE WHEN lower(C.TYPE_NAME) like 'varchar%' THEN cast(regexp_extract(upper(C.TYPE_NAME), '^VARCHAR\\s*\\((\\d+)\\s*\\)$', 1) as int) + WHEN lower(C.TYPE_NAME) like 'char%' THEN cast(regexp_extract(upper(C.TYPE_NAME), '^CHAR\\s*\\((\\d+)\\s*\\)$', 1) as int) + ELSE null END, + CASE WHEN lower(C.TYPE_NAME) like 'varchar%' THEN cast(regexp_extract(upper(C.TYPE_NAME), '^VARCHAR\\s*\\((\\d+)\\s*\\)$', 1) as int) + WHEN lower(C.TYPE_NAME) like 'char%' THEN cast(regexp_extract(upper(C.TYPE_NAME), '^CHAR\\s*\\((\\d+)\\s*\\)$', 1) as int) + ELSE null END, + CASE WHEN lower(C.TYPE_NAME) = 'bigint' THEN 19 + WHEN lower(C.TYPE_NAME) = 'int' THEN 10 + WHEN lower(C.TYPE_NAME) = 'smallint' THEN 5 + WHEN lower(C.TYPE_NAME) = 'tinyint' THEN 3 + WHEN lower(C.TYPE_NAME) = 'float' THEN 23 + WHEN lower(C.TYPE_NAME) = 'double' THEN 53 + WHEN lower(C.TYPE_NAME) like 'decimal%' THEN regexp_extract(upper(C.TYPE_NAME), '^DECIMAL\\s*\\((\\d+)',1) + WHEN lower(C.TYPE_NAME) like 'numeric%' THEN regexp_extract(upper(C.TYPE_NAME), '^NUMERIC\\s*\\((\\d+)',1) + ELSE null END, + CASE WHEN lower(C.TYPE_NAME) = 'bigint' THEN 10 + WHEN lower(C.TYPE_NAME) = 'int' THEN 10 + WHEN lower(C.TYPE_NAME) = 'smallint' THEN 10 + WHEN lower(C.TYPE_NAME) = 'tinyint' THEN 10 + WHEN lower(C.TYPE_NAME) = 'float' THEN 2 + WHEN lower(C.TYPE_NAME) = 'double' THEN 2 + WHEN lower(C.TYPE_NAME) like 'decimal%' THEN 10 + WHEN lower(C.TYPE_NAME) like 'numeric%' THEN 10 + ELSE null END, + CASE WHEN lower(C.TYPE_NAME) like 'decimal%' THEN regexp_extract(upper(C.TYPE_NAME), '^DECIMAL\\s*\\((\\d+),(\\d+)',2) + WHEN lower(C.TYPE_NAME) like 'numeric%' THEN regexp_extract(upper(C.TYPE_NAME), '^NUMERIC\\s*\\((\\d+),(\\d+)',2) + ELSE null END, + CASE WHEN lower(C.TYPE_NAME) = 'date' THEN 0 + WHEN lower(C.TYPE_NAME) = 'timestamp' THEN 9 + ELSE null END, + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + C.CD_ID, + 'NO', + 'NO', + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + cast (null as string), + 'NEVER', + cast (null as string), + 'NO', + 'NO', + cast (null as string), + 'YES', + C.TYPE_NAME as DECLARED_DATA_TYPE, + CASE WHEN lower(C.TYPE_NAME) = 'bigint' THEN 19 + WHEN lower(C.TYPE_NAME) = 'int' THEN 10 + WHEN lower(C.TYPE_NAME) = 'smallint' THEN 5 + WHEN lower(C.TYPE_NAME) = 'tinyint' THEN 3 + WHEN lower(C.TYPE_NAME) = 'float' THEN 23 + WHEN lower(C.TYPE_NAME) = 'double' THEN 53 + WHEN lower(C.TYPE_NAME) like 'decimal%' THEN regexp_extract(upper(C.TYPE_NAME), '^DECIMAL\\s*\\((\\d+)',1) + WHEN lower(C.TYPE_NAME) like 'numeric%' THEN regexp_extract(upper(C.TYPE_NAME), '^NUMERIC\\s*\\((\\d+)',1) + ELSE null END, + CASE WHEN lower(C.TYPE_NAME) = 'bigint' THEN 10 + WHEN lower(C.TYPE_NAME) = 'int' THEN 10 + WHEN lower(C.TYPE_NAME) = 'smallint' THEN 10 + WHEN lower(C.TYPE_NAME) = 'tinyint' THEN 10 + WHEN lower(C.TYPE_NAME) = 'float' THEN 2 + WHEN lower(C.TYPE_NAME) = 'double' THEN 2 + WHEN lower(C.TYPE_NAME) like 'decimal%' THEN 10 + WHEN lower(C.TYPE_NAME) like 'numeric%' THEN 10 + ELSE null END +FROM + sys.`COLUMNS_V2` C, + sys.`SDS` S, + sys.`TBLS` T, + sys.`DBS` D +WHERE + S.`SD_ID` = T.`SD_ID` + AND T.`DB_ID` = D.`DB_ID` + AND C.`CD_ID` = S.`CD_ID` +POSTHOOK: type: CREATEVIEW +POSTHOOK: Input: sys@columns_v2 +POSTHOOK: Input: sys@dbs +POSTHOOK: Input: sys@sds +POSTHOOK: Input: sys@tbls +POSTHOOK: Output: INFORMATION_SCHEMA@COLUMNS +POSTHOOK: Output: database:information_schema +POSTHOOK: Lineage: COLUMNS.character_maximum_length EXPRESSION [(columns_v2)c.FieldSchema(name:type_name, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: COLUMNS.character_octet_length EXPRESSION [(columns_v2)c.FieldSchema(name:type_name, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: COLUMNS.character_set_catalog EXPRESSION [] +POSTHOOK: Lineage: COLUMNS.character_set_name EXPRESSION [] +POSTHOOK: Lineage: COLUMNS.character_set_schema EXPRESSION [] +POSTHOOK: Lineage: COLUMNS.collation_catalog EXPRESSION [] +POSTHOOK: Lineage: COLUMNS.collation_name EXPRESSION [] +POSTHOOK: Lineage: COLUMNS.collation_schema EXPRESSION [] +POSTHOOK: Lineage: COLUMNS.column_default EXPRESSION [] +POSTHOOK: Lineage: COLUMNS.column_name SIMPLE [(columns_v2)c.FieldSchema(name:column_name, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: COLUMNS.data_type SIMPLE [(columns_v2)c.FieldSchema(name:type_name, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: COLUMNS.datetime_precision EXPRESSION [(columns_v2)c.FieldSchema(name:type_name, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: COLUMNS.declared_data_type SIMPLE [(columns_v2)c.FieldSchema(name:type_name, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: COLUMNS.declared_numeric_precision EXPRESSION [(columns_v2)c.FieldSchema(name:type_name, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: COLUMNS.declared_numeric_scale EXPRESSION [(columns_v2)c.FieldSchema(name:type_name, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: COLUMNS.dtd_identifier SIMPLE [(columns_v2)c.FieldSchema(name:cd_id, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: COLUMNS.generation_expression EXPRESSION [] +POSTHOOK: Lineage: COLUMNS.identity_cycle EXPRESSION [] +POSTHOOK: Lineage: COLUMNS.identity_generation EXPRESSION [] +POSTHOOK: Lineage: COLUMNS.identity_increment EXPRESSION [] +POSTHOOK: Lineage: COLUMNS.identity_maximum EXPRESSION [] +POSTHOOK: Lineage: COLUMNS.identity_minimum EXPRESSION [] +POSTHOOK: Lineage: COLUMNS.identity_start EXPRESSION [] +POSTHOOK: Lineage: COLUMNS.interval_precision EXPRESSION [] +POSTHOOK: Lineage: COLUMNS.interval_type EXPRESSION [] +POSTHOOK: Lineage: COLUMNS.is_generated SIMPLE [] +POSTHOOK: Lineage: COLUMNS.is_identity SIMPLE [] +POSTHOOK: Lineage: COLUMNS.is_nullable SIMPLE [] +POSTHOOK: Lineage: COLUMNS.is_self_referencing SIMPLE [] +POSTHOOK: Lineage: COLUMNS.is_system_time_period_end SIMPLE [] +POSTHOOK: Lineage: COLUMNS.is_system_time_period_start SIMPLE [] +POSTHOOK: Lineage: COLUMNS.is_updatable SIMPLE [] +POSTHOOK: Lineage: COLUMNS.maximum_cardinality EXPRESSION [] +POSTHOOK: Lineage: COLUMNS.numeric_precision EXPRESSION [(columns_v2)c.FieldSchema(name:type_name, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: COLUMNS.numeric_precision_radix EXPRESSION [(columns_v2)c.FieldSchema(name:type_name, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: COLUMNS.numeric_scale EXPRESSION [(columns_v2)c.FieldSchema(name:type_name, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: COLUMNS.ordinal_position SIMPLE [(columns_v2)c.FieldSchema(name:integer_idx, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: COLUMNS.scope_catalog EXPRESSION [] +POSTHOOK: Lineage: COLUMNS.scope_name EXPRESSION [] +POSTHOOK: Lineage: COLUMNS.scope_schema EXPRESSION [] +POSTHOOK: Lineage: COLUMNS.system_time_period_timestamp_generation EXPRESSION [] +POSTHOOK: Lineage: COLUMNS.table_catalog SIMPLE [] +POSTHOOK: Lineage: COLUMNS.table_name SIMPLE [(tbls)t.FieldSchema(name:tbl_name, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: COLUMNS.table_schema SIMPLE [(dbs)d.FieldSchema(name:name, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: COLUMNS.udt_catalog EXPRESSION [] +POSTHOOK: Lineage: COLUMNS.udt_name EXPRESSION [] +POSTHOOK: Lineage: COLUMNS.udt_schema EXPRESSION [] +PREHOOK: query: CREATE VIEW IF NOT EXISTS `COLUMN_PRIVILEGES` +( + `GRANTOR`, + `GRANTEE`, + `TABLE_CATALOG`, + `TABLE_SCHEMA`, + `TABLE_NAME`, + `COLUMN_NAME`, + `PRIVILEGE_TYPE`, + `IS_GRANTABLE` +) AS +SELECT + `GRANTOR`, + `PRINCIPAL_NAME`, + 'default', + D.`NAME`, + T.`TBL_NAME`, + C.`COLUMN_NAME`, + P.`TBL_COL_PRIV`, + IF (P.`GRANT_OPTION` == 0, 'NO', 'YES') +FROM + sys.`TBL_COL_PRIVS` P, + sys.`TBLS` T, + sys.`DBS` D, + sys.`COLUMNS_V2` C, + sys.`SDS` S +WHERE + S.`SD_ID` = T.`SD_ID` + AND T.`DB_ID` = D.`DB_ID` + AND P.`TBL_ID` = T.`TBL_ID` + AND P.`COLUMN_NAME` = C.`COLUMN_NAME` + AND C.`CD_ID` = S.`CD_ID` +PREHOOK: type: CREATEVIEW +PREHOOK: Input: sys@columns_v2 +PREHOOK: Input: sys@dbs +PREHOOK: Input: sys@sds +PREHOOK: Input: sys@tbl_col_privs +PREHOOK: Input: sys@tbls +PREHOOK: Output: INFORMATION_SCHEMA@COLUMN_PRIVILEGES +PREHOOK: Output: database:information_schema +POSTHOOK: query: CREATE VIEW IF NOT EXISTS `COLUMN_PRIVILEGES` +( + `GRANTOR`, + `GRANTEE`, + `TABLE_CATALOG`, + `TABLE_SCHEMA`, + `TABLE_NAME`, + `COLUMN_NAME`, + `PRIVILEGE_TYPE`, + `IS_GRANTABLE` +) AS +SELECT + `GRANTOR`, + `PRINCIPAL_NAME`, + 'default', + D.`NAME`, + T.`TBL_NAME`, + C.`COLUMN_NAME`, + P.`TBL_COL_PRIV`, + IF (P.`GRANT_OPTION` == 0, 'NO', 'YES') +FROM + sys.`TBL_COL_PRIVS` P, + sys.`TBLS` T, + sys.`DBS` D, + sys.`COLUMNS_V2` C, + sys.`SDS` S +WHERE + S.`SD_ID` = T.`SD_ID` + AND T.`DB_ID` = D.`DB_ID` + AND P.`TBL_ID` = T.`TBL_ID` + AND P.`COLUMN_NAME` = C.`COLUMN_NAME` + AND C.`CD_ID` = S.`CD_ID` +POSTHOOK: type: CREATEVIEW +POSTHOOK: Input: sys@columns_v2 +POSTHOOK: Input: sys@dbs +POSTHOOK: Input: sys@sds +POSTHOOK: Input: sys@tbl_col_privs +POSTHOOK: Input: sys@tbls +POSTHOOK: Output: INFORMATION_SCHEMA@COLUMN_PRIVILEGES +POSTHOOK: Output: database:information_schema +POSTHOOK: Lineage: COLUMN_PRIVILEGES.column_name SIMPLE [(columns_v2)c.FieldSchema(name:column_name, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: COLUMN_PRIVILEGES.grantee SIMPLE [(tbl_col_privs)p.FieldSchema(name:principal_name, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: COLUMN_PRIVILEGES.grantor SIMPLE [(tbl_col_privs)p.FieldSchema(name:grantor, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: COLUMN_PRIVILEGES.is_grantable EXPRESSION [(tbl_col_privs)p.FieldSchema(name:grant_option, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: COLUMN_PRIVILEGES.privilege_type SIMPLE [(tbl_col_privs)p.FieldSchema(name:tbl_col_priv, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: COLUMN_PRIVILEGES.table_catalog SIMPLE [] +POSTHOOK: Lineage: COLUMN_PRIVILEGES.table_name SIMPLE [(tbls)t.FieldSchema(name:tbl_name, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: COLUMN_PRIVILEGES.table_schema SIMPLE [(dbs)d.FieldSchema(name:name, type:string, comment:from deserializer), ] +PREHOOK: query: CREATE VIEW IF NOT EXISTS `VIEWS` +( + `TABLE_CATALOG`, + `TABLE_SCHEMA`, + `TABLE_NAME`, + `VIEW_DEFINITION`, + `CHECK_OPTION`, + `IS_UPDATABLE`, + `IS_INSERTABLE_INTO`, + `IS_TRIGGER_UPDATABLE`, + `IS_TRIGGER_DELETABLE`, + `IS_TRIGGER_INSERTABLE_INTO` +) AS +SELECT + 'default', + D.NAME, + T.TBL_NAME, + T.VIEW_ORIGINAL_TEXT, + CAST(NULL as string), + false, + false, + false, + false, + false +FROM + `sys`.`DBS` D, + `sys`.`TBLS` T +WHERE + D.`DB_ID` = T.`DB_ID` AND + length(T.VIEW_ORIGINAL_TEXT) > 0 +PREHOOK: type: CREATEVIEW +PREHOOK: Input: sys@dbs +PREHOOK: Input: sys@tbls +PREHOOK: Output: INFORMATION_SCHEMA@VIEWS +PREHOOK: Output: database:information_schema +POSTHOOK: query: CREATE VIEW IF NOT EXISTS `VIEWS` +( + `TABLE_CATALOG`, + `TABLE_SCHEMA`, + `TABLE_NAME`, + `VIEW_DEFINITION`, + `CHECK_OPTION`, + `IS_UPDATABLE`, + `IS_INSERTABLE_INTO`, + `IS_TRIGGER_UPDATABLE`, + `IS_TRIGGER_DELETABLE`, + `IS_TRIGGER_INSERTABLE_INTO` +) AS +SELECT + 'default', + D.NAME, + T.TBL_NAME, + T.VIEW_ORIGINAL_TEXT, + CAST(NULL as string), + false, + false, + false, + false, + false +FROM + `sys`.`DBS` D, + `sys`.`TBLS` T +WHERE + D.`DB_ID` = T.`DB_ID` AND + length(T.VIEW_ORIGINAL_TEXT) > 0 +POSTHOOK: type: CREATEVIEW +POSTHOOK: Input: sys@dbs +POSTHOOK: Input: sys@tbls +POSTHOOK: Output: INFORMATION_SCHEMA@VIEWS +POSTHOOK: Output: database:information_schema +POSTHOOK: Lineage: VIEWS.check_option EXPRESSION [] +POSTHOOK: Lineage: VIEWS.is_insertable_into SIMPLE [] +POSTHOOK: Lineage: VIEWS.is_trigger_deletable SIMPLE [] +POSTHOOK: Lineage: VIEWS.is_trigger_insertable_into SIMPLE [] +POSTHOOK: Lineage: VIEWS.is_trigger_updatable SIMPLE [] +POSTHOOK: Lineage: VIEWS.is_updatable SIMPLE [] +POSTHOOK: Lineage: VIEWS.table_catalog SIMPLE [] +POSTHOOK: Lineage: VIEWS.table_name SIMPLE [(tbls)t.FieldSchema(name:tbl_name, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: VIEWS.table_schema SIMPLE [(dbs)d.FieldSchema(name:name, type:string, comment:from deserializer), ] +POSTHOOK: Lineage: VIEWS.view_definition SIMPLE [(tbls)t.FieldSchema(name:view_original_text, type:string, comment:from deserializer), ] +PREHOOK: query: use sys +PREHOOK: type: SWITCHDATABASE +PREHOOK: Input: database:sys +POSTHOOK: query: use sys +POSTHOOK: type: SWITCHDATABASE +POSTHOOK: Input: database:sys +PREHOOK: query: select bucket_col_name, integer_idx from bucketing_cols order by bucket_col_name, integer_idx limit 5 +PREHOOK: type: QUERY +PREHOOK: Input: sys@bucketing_cols +#### A masked pattern was here #### +POSTHOOK: query: select bucket_col_name, integer_idx from bucketing_cols order by bucket_col_name, integer_idx limit 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@bucketing_cols +#### A masked pattern was here #### +key 0 +key 0 +key 0 +value 0 +PREHOOK: query: select count(*) from cds +PREHOOK: type: QUERY +PREHOOK: Input: sys@cds +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from cds +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@cds +#### A masked pattern was here #### +63 +PREHOOK: query: select column_name, type_name, integer_idx from columns_v2 order by column_name, integer_idx limit 5 +PREHOOK: type: QUERY +PREHOOK: Input: sys@columns_v2 +#### A masked pattern was here #### +POSTHOOK: query: select column_name, type_name, integer_idx from columns_v2 order by column_name, integer_idx limit 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@columns_v2 +#### A masked pattern was here #### +a decimal(10,2) 0 +add_time int 1 +aint int 0 +astring string 1 +attributes map,map>>>> 6 +PREHOOK: query: select param_key, param_value from database_params order by param_key, param_value limit 5 +PREHOOK: type: QUERY +PREHOOK: Input: sys@database_params +#### A masked pattern was here #### +POSTHOOK: query: select param_key, param_value from database_params order by param_key, param_value limit 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@database_params +#### A masked pattern was here #### +PREHOOK: type: QUERY +PREHOOK: Input: sys@dbs +#### A masked pattern was here #### +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@dbs +#### A masked pattern was here #### +PREHOOK: query: select grantor, principal_name from db_privs order by grantor, principal_name limit 5 +PREHOOK: type: QUERY +PREHOOK: Input: sys@db_privs +#### A masked pattern was here #### +POSTHOOK: query: select grantor, principal_name from db_privs order by grantor, principal_name limit 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@db_privs +#### A masked pattern was here #### +PREHOOK: query: select grantor, principal_name from global_privs order by grantor, principal_name limit 5 +PREHOOK: type: QUERY +PREHOOK: Input: sys@global_privs +#### A masked pattern was here #### +POSTHOOK: query: select grantor, principal_name from global_privs order by grantor, principal_name limit 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@global_privs +#### A masked pattern was here #### +admin admin +PREHOOK: query: select index_name, index_handler_class from idxs order by index_name limit 5 +PREHOOK: type: QUERY +PREHOOK: Input: sys@idxs +#### A masked pattern was here #### +POSTHOOK: query: select index_name, index_handler_class from idxs order by index_name limit 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@idxs +#### A masked pattern was here #### +PREHOOK: query: select param_key, param_value from index_params order by param_key, param_value limit 5 +PREHOOK: type: QUERY +PREHOOK: Input: sys@index_params +#### A masked pattern was here #### +POSTHOOK: query: select param_key, param_value from index_params order by param_key, param_value limit 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@index_params +#### A masked pattern was here #### +PREHOOK: query: select part_name from partitions order by part_name limit 5 +PREHOOK: type: QUERY +PREHOOK: Input: sys@partitions +#### A masked pattern was here #### +POSTHOOK: query: select part_name from partitions order by part_name limit 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@partitions +#### A masked pattern was here #### +ds=2008-04-08/hr=11 +ds=2008-04-08/hr=12 +ds=2008-04-09/hr=11 +ds=2008-04-09/hr=12 +dt=2014 +PREHOOK: query: select pkey_name, pkey_type from partition_keys order by pkey_name limit 5 +PREHOOK: type: QUERY +PREHOOK: Input: sys@partition_keys +#### A masked pattern was here #### +POSTHOOK: query: select pkey_name, pkey_type from partition_keys order by pkey_name limit 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@partition_keys +#### A masked pattern was here #### +ds string +dt string +dt string +hr string +PREHOOK: query: select part_key_val, integer_idx from partition_key_vals order by part_key_val, integer_idx limit 5 +PREHOOK: type: QUERY +PREHOOK: Input: sys@partition_key_vals +#### A masked pattern was here #### +POSTHOOK: query: select part_key_val, integer_idx from partition_key_vals order by part_key_val, integer_idx limit 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@partition_key_vals +#### A masked pattern was here #### +11 1 +11 1 +12 1 +12 1 +2008-04-08 0 +PREHOOK: query: select param_key, param_value from partition_params order by param_key, param_value limit 5 +PREHOOK: type: QUERY +PREHOOK: Input: sys@partition_params +#### A masked pattern was here #### +POSTHOOK: query: select param_key, param_value from partition_params order by param_key, param_value limit 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@partition_params +#### A masked pattern was here #### +COLUMN_STATS_ACCURATE {"BASIC_STATS":"true","COLUMN_STATS":{"c_boolean":"true","c_float":"true","c_int":"true","key":"true","value":"true"}} +COLUMN_STATS_ACCURATE {"BASIC_STATS":"true","COLUMN_STATS":{"c_boolean":"true","c_float":"true","c_int":"true","key":"true","value":"true"}} +COLUMN_STATS_ACCURATE {"BASIC_STATS":"true","COLUMN_STATS":{"key":"true","value":"true"}} +COLUMN_STATS_ACCURATE {"BASIC_STATS":"true","COLUMN_STATS":{"key":"true","value":"true"}} +COLUMN_STATS_ACCURATE {"BASIC_STATS":"true","COLUMN_STATS":{"key":"true","value":"true"}} +PREHOOK: query: select grantor, principal_name from part_col_privs order by grantor, principal_name limit 5 +PREHOOK: type: QUERY +PREHOOK: Input: sys@part_col_privs +#### A masked pattern was here #### +POSTHOOK: query: select grantor, principal_name from part_col_privs order by grantor, principal_name limit 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@part_col_privs +#### A masked pattern was here #### +PREHOOK: query: select grantor, principal_name from part_privs order by grantor, principal_name limit 5 +PREHOOK: type: QUERY +PREHOOK: Input: sys@part_privs +#### A masked pattern was here #### +POSTHOOK: query: select grantor, principal_name from part_privs order by grantor, principal_name limit 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@part_privs +#### A masked pattern was here #### +PREHOOK: query: select role_name from roles order by role_name limit 5 +PREHOOK: type: QUERY +PREHOOK: Input: sys@roles +#### A masked pattern was here #### +POSTHOOK: query: select role_name from roles order by role_name limit 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@roles +#### A masked pattern was here #### +admin +public +PREHOOK: query: select principal_name, grantor from role_map order by principal_name, grantor limit 5 +PREHOOK: type: QUERY +PREHOOK: Input: sys@role_map +#### A masked pattern was here #### +POSTHOOK: query: select principal_name, grantor from role_map order by principal_name, grantor limit 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@role_map +#### A masked pattern was here #### +PREHOOK: query: select count(*) from sds +PREHOOK: type: QUERY +PREHOOK: Input: sys@sds +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from sds +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@sds +#### A masked pattern was here #### +69 +PREHOOK: query: select param_key, param_value from sd_params order by param_key, param_value limit 5 +PREHOOK: type: QUERY +PREHOOK: Input: sys@sd_params +#### A masked pattern was here #### +POSTHOOK: query: select param_key, param_value from sd_params order by param_key, param_value limit 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@sd_params +#### A masked pattern was here #### +PREHOOK: query: select sequence_name from sequence_table order by sequence_name limit 5 +PREHOOK: type: QUERY +PREHOOK: Input: sys@sequence_table +#### A masked pattern was here #### +POSTHOOK: query: select sequence_name from sequence_table order by sequence_name limit 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@sequence_table +#### A masked pattern was here #### +org.apache.hadoop.hive.metastore.model.MColumnDescriptor +org.apache.hadoop.hive.metastore.model.MDatabase +org.apache.hadoop.hive.metastore.model.MFunction +org.apache.hadoop.hive.metastore.model.MGlobalPrivilege +org.apache.hadoop.hive.metastore.model.MPartition +PREHOOK: query: select name, slib from serdes order by name, slib limit 5 +PREHOOK: type: QUERY +PREHOOK: Input: sys@serdes +#### A masked pattern was here #### +POSTHOOK: query: select name, slib from serdes order by name, slib limit 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@serdes +#### A masked pattern was here #### +NULL NULL +NULL NULL +NULL NULL +NULL NULL +NULL NULL +PREHOOK: query: select param_key, param_value from serde_params order by param_key, param_value limit 5 +PREHOOK: type: QUERY +PREHOOK: Input: sys@serde_params +#### A masked pattern was here #### +POSTHOOK: query: select param_key, param_value from serde_params order by param_key, param_value limit 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@serde_params +#### A masked pattern was here #### +field.delim , +field.delim , +field.delim , +field.delim , +field.delim , +PREHOOK: query: select skewed_col_name from skewed_col_names order by skewed_col_name limit 5 +PREHOOK: type: QUERY +PREHOOK: Input: sys@skewed_col_names +#### A masked pattern was here #### +POSTHOOK: query: select skewed_col_name from skewed_col_names order by skewed_col_name limit 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@skewed_col_names +#### A masked pattern was here #### +key +PREHOOK: query: select count(*) from skewed_col_value_loc_map +PREHOOK: type: QUERY +PREHOOK: Input: sys@skewed_col_value_loc_map +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from skewed_col_value_loc_map +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@skewed_col_value_loc_map +#### A masked pattern was here #### +0 +PREHOOK: query: select count(*) from skewed_string_list +PREHOOK: type: QUERY +PREHOOK: Input: sys@skewed_string_list +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from skewed_string_list +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@skewed_string_list +#### A masked pattern was here #### +3 +PREHOOK: query: select count(*) from skewed_string_list_values +PREHOOK: type: QUERY +PREHOOK: Input: sys@skewed_string_list_values +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from skewed_string_list_values +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@skewed_string_list_values +#### A masked pattern was here #### +3 +PREHOOK: query: select count(*) from skewed_values +PREHOOK: type: QUERY +PREHOOK: Input: sys@skewed_values +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from skewed_values +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@skewed_values +#### A masked pattern was here #### +3 +PREHOOK: query: select column_name, `order` from sort_cols order by column_name limit 5 +PREHOOK: type: QUERY +PREHOOK: Input: sys@sort_cols +#### A masked pattern was here #### +POSTHOOK: query: select column_name, `order` from sort_cols order by column_name limit 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@sort_cols +#### A masked pattern was here #### +PREHOOK: query: select param_key, param_value from table_params order by param_key, param_value limit 5 +PREHOOK: type: QUERY +PREHOOK: Input: sys@table_params +#### A masked pattern was here #### +POSTHOOK: query: select param_key, param_value from table_params order by param_key, param_value limit 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@table_params +#### A masked pattern was here #### +COLUMN_STATS_ACCURATE {"BASIC_STATS":"true","COLUMN_STATS":{"c_boolean":"true","c_float":"true","c_int":"true","key":"true","value":"true"}} +COLUMN_STATS_ACCURATE {"BASIC_STATS":"true","COLUMN_STATS":{"cbigint":"true","cboolean1":"true","cboolean2":"true","cdouble":"true","cfloat":"true","cint":"true","csmallint":"true","cstring1":"true","cstring2":"true","ctimestamp1":"true","ctimestamp2":"true","ctinyint":"true"}} +COLUMN_STATS_ACCURATE {"BASIC_STATS":"true","COLUMN_STATS":{"json":"true"}} +COLUMN_STATS_ACCURATE {"BASIC_STATS":"true","COLUMN_STATS":{"key":"true","value":"true"}} +COLUMN_STATS_ACCURATE {"BASIC_STATS":"true","COLUMN_STATS":{"key":"true","value":"true"}} +PREHOOK: query: select tbl_name from tbls order by tbl_name limit 5 +PREHOOK: type: QUERY +PREHOOK: Input: sys@tbls +#### A masked pattern was here #### +POSTHOOK: query: select tbl_name from tbls order by tbl_name limit 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@tbls +#### A masked pattern was here #### +alltypesorc +bucketing_cols +cbo_t1 +cbo_t2 +cbo_t3 +PREHOOK: query: select column_name, grantor, principal_name from tbl_col_privs order by column_name, principal_name limit 5 +PREHOOK: type: QUERY +PREHOOK: Input: sys@tbl_col_privs +#### A masked pattern was here #### +POSTHOOK: query: select column_name, grantor, principal_name from tbl_col_privs order by column_name, principal_name limit 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@tbl_col_privs +#### A masked pattern was here #### +PREHOOK: query: select grantor, principal_name from tbl_privs order by grantor, principal_name limit 5 +PREHOOK: type: QUERY +PREHOOK: Input: sys@tbl_privs +#### A masked pattern was here #### +POSTHOOK: query: select grantor, principal_name from tbl_privs order by grantor, principal_name limit 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@tbl_privs +#### A masked pattern was here #### +hive_test_user hive_test_user +hive_test_user hive_test_user +hive_test_user hive_test_user +hive_test_user hive_test_user +hive_test_user hive_test_user +PREHOOK: query: select table_name, column_name, num_nulls, num_distincts from tab_col_stats order by table_name, column_name limit 10 +PREHOOK: type: QUERY +PREHOOK: Input: sys@tab_col_stats +#### A masked pattern was here #### +POSTHOOK: query: select table_name, column_name, num_nulls, num_distincts from tab_col_stats order by table_name, column_name limit 10 +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@tab_col_stats +#### A masked pattern was here #### +alltypesorc cbigint 3115 4083 +alltypesorc cboolean1 3114 NULL +alltypesorc cboolean2 3115 NULL +alltypesorc cdouble 3114 4264 +alltypesorc cfloat 3115 117 +alltypesorc cint 3115 5774 +alltypesorc csmallint 3114 5529 +alltypesorc cstring1 3114 7488 +alltypesorc cstring2 3115 4083 +alltypesorc ctimestamp1 3115 31 +PREHOOK: query: select table_name, partition_name, column_name, num_nulls, num_distincts from part_col_stats order by table_name, partition_name, column_name limit 10 +PREHOOK: type: QUERY +PREHOOK: Input: sys@part_col_stats +#### A masked pattern was here #### +POSTHOOK: query: select table_name, partition_name, column_name, num_nulls, num_distincts from part_col_stats order by table_name, partition_name, column_name limit 10 +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@part_col_stats +#### A masked pattern was here #### +cbo_t1 dt=2014 c_boolean 2 NULL +cbo_t1 dt=2014 c_float 2 1 +cbo_t1 dt=2014 c_int 2 1 +cbo_t1 dt=2014 key 2 4 +cbo_t1 dt=2014 value 2 4 +cbo_t2 dt=2014 c_boolean 2 NULL +cbo_t2 dt=2014 c_float 2 2 +cbo_t2 dt=2014 c_int 2 2 +cbo_t2 dt=2014 key 2 4 +cbo_t2 dt=2014 value 2 4 +PREHOOK: query: select schema_version from version order by schema_version limit 5 +PREHOOK: type: QUERY +PREHOOK: Input: sys@version +#### A masked pattern was here #### +POSTHOOK: query: select schema_version from version order by schema_version limit 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@version +#### A masked pattern was here #### +PREHOOK: query: select func_name, func_type from funcs order by func_name, func_type limit 5 +PREHOOK: type: QUERY +PREHOOK: Input: sys@funcs +#### A masked pattern was here #### +POSTHOOK: query: select func_name, func_type from funcs order by func_name, func_type limit 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@funcs +#### A masked pattern was here #### +qtest_get_java_boolean 1 +PREHOOK: query: select constraint_name from key_constraints order by constraint_name limit 5 +PREHOOK: type: QUERY +PREHOOK: Input: sys@key_constraints +#### A masked pattern was here #### +POSTHOOK: query: select constraint_name from key_constraints order by constraint_name limit 5 +POSTHOOK: type: QUERY +POSTHOOK: Input: sys@key_constraints +#### A masked pattern was here #### +sys_pk_bucketing_cols +sys_pk_bucketing_cols +sys_pk_cds +sys_pk_col_value_loc_map +sys_pk_col_value_loc_map +PREHOOK: query: use INFORMATION_SCHEMA +PREHOOK: type: SWITCHDATABASE +PREHOOK: Input: database:information_schema +POSTHOOK: query: use INFORMATION_SCHEMA +POSTHOOK: type: SWITCHDATABASE +POSTHOOK: Input: database:information_schema +PREHOOK: query: select count(*) from SCHEMATA +PREHOOK: type: QUERY +PREHOOK: Input: information_schema@schemata +PREHOOK: Input: sys@dbs +#### A masked pattern was here #### +POSTHOOK: query: select count(*) from SCHEMATA +POSTHOOK: type: QUERY +POSTHOOK: Input: information_schema@schemata +POSTHOOK: Input: sys@dbs +#### A masked pattern was here #### +3 +PREHOOK: query: select * from TABLES order by TABLE_SCHEMA, TABLE_NAME +PREHOOK: type: QUERY +PREHOOK: Input: information_schema@tables +PREHOOK: Input: sys@dbs +PREHOOK: Input: sys@tbls +#### A masked pattern was here #### +POSTHOOK: query: select * from TABLES order by TABLE_SCHEMA, TABLE_NAME +POSTHOOK: type: QUERY +POSTHOOK: Input: information_schema@tables +POSTHOOK: Input: sys@dbs +POSTHOOK: Input: sys@tbls +#### A masked pattern was here #### +default default alltypesorc BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default default cbo_t1 BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default default cbo_t2 BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default default cbo_t3 BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default default lineitem BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default default moretypes BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default default part BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default default scr_txn BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default default src BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default default src1 BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default default src_buck BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default default src_cbo BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default default src_json BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default default src_sequencefile BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default default src_skew BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default default src_thrift BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default default srcbucket BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default default srcbucket2 BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default default srcpart BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default information_schema column_privileges VIEW NULL NULL NULL NULL NULL NO NO NULL +default information_schema columns VIEW NULL NULL NULL NULL NULL NO NO NULL +default information_schema schemata VIEW NULL NULL NULL NULL NULL NO NO NULL +default information_schema table_privileges VIEW NULL NULL NULL NULL NULL NO NO NULL +default information_schema tables VIEW NULL NULL NULL NULL NULL NO NO NULL +default information_schema views VIEW NULL NULL NULL NULL NULL NO NO NULL +default sys bucketing_cols BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys cds BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys columns_v2 BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys database_params BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys db_privs BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys db_version BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys dbs BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys funcs BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys global_privs BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys idxs BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys index_params BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys key_constraints BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys part_col_privs BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys part_col_stats BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys part_privs BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys partition_key_vals BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys partition_keys BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys partition_params BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys partitions BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys role_map BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys roles BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys sd_params BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys sds BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys sequence_table BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys serde_params BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys serdes BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys skewed_col_names BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys skewed_col_value_loc_map BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys skewed_string_list BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys skewed_string_list_values BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys skewed_values BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys sort_cols BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys tab_col_stats BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys table_params BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys tbl_col_privs BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys tbl_privs BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys tbls BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +default sys version BASE_TABLE NULL NULL NULL NULL NULL YES NO NULL +PREHOOK: query: select * from TABLE_PRIVILEGES order by GRANTOR, GRANTEE, TABLE_SCHEMA, TABLE_NAME limit 10 +PREHOOK: type: QUERY +PREHOOK: Input: information_schema@table_privileges +PREHOOK: Input: sys@dbs +PREHOOK: Input: sys@tbl_privs +PREHOOK: Input: sys@tbls +#### A masked pattern was here #### +POSTHOOK: query: select * from TABLE_PRIVILEGES order by GRANTOR, GRANTEE, TABLE_SCHEMA, TABLE_NAME limit 10 +POSTHOOK: type: QUERY +POSTHOOK: Input: information_schema@table_privileges +POSTHOOK: Input: sys@dbs +POSTHOOK: Input: sys@tbl_privs +POSTHOOK: Input: sys@tbls +#### A masked pattern was here #### +hive_test_user hive_test_user default default alltypesorc DELETE YES NO +hive_test_user hive_test_user default default alltypesorc INSERT YES NO +hive_test_user hive_test_user default default alltypesorc SELECT YES NO +hive_test_user hive_test_user default default alltypesorc UPDATE YES NO +hive_test_user hive_test_user default default cbo_t1 SELECT YES NO +hive_test_user hive_test_user default default cbo_t1 UPDATE YES NO +hive_test_user hive_test_user default default cbo_t1 INSERT YES NO +hive_test_user hive_test_user default default cbo_t1 DELETE YES NO +hive_test_user hive_test_user default default cbo_t2 UPDATE YES NO +hive_test_user hive_test_user default default cbo_t2 SELECT YES NO +PREHOOK: query: select * from COLUMNS where TABLE_NAME = 'alltypesorc' or TABLE_NAME = 'moretypes' order by TABLE_SCHEMA, TABLE_NAME, ORDINAL_POSITION +PREHOOK: type: QUERY +PREHOOK: Input: information_schema@columns +PREHOOK: Input: sys@columns_v2 +PREHOOK: Input: sys@dbs +PREHOOK: Input: sys@sds +PREHOOK: Input: sys@tbls +#### A masked pattern was here #### +POSTHOOK: query: select * from COLUMNS where TABLE_NAME = 'alltypesorc' or TABLE_NAME = 'moretypes' order by TABLE_SCHEMA, TABLE_NAME, ORDINAL_POSITION +POSTHOOK: type: QUERY +POSTHOOK: Input: information_schema@columns +POSTHOOK: Input: sys@columns_v2 +POSTHOOK: Input: sys@dbs +POSTHOOK: Input: sys@sds +POSTHOOK: Input: sys@tbls +#### A masked pattern was here #### +default default alltypesorc ctinyint 0 NULL YES tinyint NULL NULL 3 10 NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL 11 NO NO NULL NULL NULL NULL NULL NULL NEVER NULL NO NO NULL YES tinyint 3 10 +default default alltypesorc csmallint 1 NULL YES smallint NULL NULL 5 10 NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL 11 NO NO NULL NULL NULL NULL NULL NULL NEVER NULL NO NO NULL YES smallint 5 10 +default default alltypesorc cboolean1 10 NULL YES boolean NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL 11 NO NO NULL NULL NULL NULL NULL NULL NEVER NULL NO NO NULL YES boolean NULL NULL +default default alltypesorc cboolean2 11 NULL YES boolean NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL 11 NO NO NULL NULL NULL NULL NULL NULL NEVER NULL NO NO NULL YES boolean NULL NULL +default default alltypesorc cint 2 NULL YES int NULL NULL 10 10 NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL 11 NO NO NULL NULL NULL NULL NULL NULL NEVER NULL NO NO NULL YES int 10 10 +default default alltypesorc cbigint 3 NULL YES bigint NULL NULL 19 10 NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL 11 NO NO NULL NULL NULL NULL NULL NULL NEVER NULL NO NO NULL YES bigint 19 10 +default default alltypesorc cfloat 4 NULL YES float NULL NULL 23 2 NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL 11 NO NO NULL NULL NULL NULL NULL NULL NEVER NULL NO NO NULL YES float 23 2 +default default alltypesorc cdouble 5 NULL YES double NULL NULL 53 2 NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL 11 NO NO NULL NULL NULL NULL NULL NULL NEVER NULL NO NO NULL YES double 53 2 +default default alltypesorc cstring1 6 NULL YES string NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL 11 NO NO NULL NULL NULL NULL NULL NULL NEVER NULL NO NO NULL YES string NULL NULL +default default alltypesorc cstring2 7 NULL YES string NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL 11 NO NO NULL NULL NULL NULL NULL NULL NEVER NULL NO NO NULL YES string NULL NULL +default default alltypesorc ctimestamp1 8 NULL YES timestamp NULL NULL NULL NULL NULL 9 NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL 11 NO NO NULL NULL NULL NULL NULL NULL NEVER NULL NO NO NULL YES timestamp NULL NULL +default default alltypesorc ctimestamp2 9 NULL YES timestamp NULL NULL NULL NULL NULL 9 NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL 11 NO NO NULL NULL NULL NULL NULL NULL NEVER NULL NO NO NULL YES timestamp NULL NULL +default default moretypes a 0 NULL YES decimal(10,2) NULL NULL 10 10 2 NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL 27 NO NO NULL NULL NULL NULL NULL NULL NEVER NULL NO NO NULL YES decimal(10,2) 10 10 +default default moretypes b 1 NULL YES tinyint NULL NULL 3 10 NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL 27 NO NO NULL NULL NULL NULL NULL NULL NEVER NULL NO NO NULL YES tinyint 3 10 +default default moretypes c 2 NULL YES smallint NULL NULL 5 10 NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL 27 NO NO NULL NULL NULL NULL NULL NULL NEVER NULL NO NO NULL YES smallint 5 10 +default default moretypes d 3 NULL YES int NULL NULL 10 10 NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL 27 NO NO NULL NULL NULL NULL NULL NULL NEVER NULL NO NO NULL YES int 10 10 +default default moretypes e 4 NULL YES bigint NULL NULL 19 10 NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL 27 NO NO NULL NULL NULL NULL NULL NULL NEVER NULL NO NO NULL YES bigint 19 10 +default default moretypes f 5 NULL YES varchar(10) 10 10 NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL 27 NO NO NULL NULL NULL NULL NULL NULL NEVER NULL NO NO NULL YES varchar(10) NULL NULL +default default moretypes g 6 NULL YES char(3) 3 3 NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL NULL 27 NO NO NULL NULL NULL NULL NULL NULL NEVER NULL NO NO NULL YES char(3) NULL NULL +PREHOOK: query: select * from COLUMN_PRIVILEGES order by GRANTOR, GRANTEE, TABLE_SCHEMA, TABLE_NAME, COLUMN_NAME limit 10 +PREHOOK: type: QUERY +PREHOOK: Input: information_schema@column_privileges +PREHOOK: Input: sys@columns_v2 +PREHOOK: Input: sys@dbs +PREHOOK: Input: sys@sds +PREHOOK: Input: sys@tbl_col_privs +PREHOOK: Input: sys@tbls +#### A masked pattern was here #### +POSTHOOK: query: select * from COLUMN_PRIVILEGES order by GRANTOR, GRANTEE, TABLE_SCHEMA, TABLE_NAME, COLUMN_NAME limit 10 +POSTHOOK: type: QUERY +POSTHOOK: Input: information_schema@column_privileges +POSTHOOK: Input: sys@columns_v2 +POSTHOOK: Input: sys@dbs +POSTHOOK: Input: sys@sds +POSTHOOK: Input: sys@tbl_col_privs +POSTHOOK: Input: sys@tbls +#### A masked pattern was here #### +PREHOOK: query: select TABLE_SCHEMA, TABLE_NAME from views order by TABLE_SCHEMA, TABLE_NAME +PREHOOK: type: QUERY +PREHOOK: Input: information_schema@views +PREHOOK: Input: sys@dbs +PREHOOK: Input: sys@tbls +#### A masked pattern was here #### +POSTHOOK: query: select TABLE_SCHEMA, TABLE_NAME from views order by TABLE_SCHEMA, TABLE_NAME +POSTHOOK: type: QUERY +POSTHOOK: Input: information_schema@views +POSTHOOK: Input: sys@dbs +POSTHOOK: Input: sys@tbls +#### A masked pattern was here #### +information_schema column_privileges +information_schema columns +information_schema schemata +information_schema table_privileges +information_schema tables +information_schema views