You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by th...@apache.org on 2014/02/18 22:07:37 UTC

svn commit: r1569527 [5/5] - in /hive/trunk: common/ common/src/java/org/apache/hadoop/hive/ant/ common/src/java/org/apache/hadoop/hive/conf/ common/src/java/org/apache/hive/common/util/ common/src/test/org/apache/hadoop/hive/conf/ conf/ itests/hive-un...

Modified: hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/jdbc/TestJdbcDriver.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/jdbc/TestJdbcDriver.java?rev=1569527&r1=1569526&r2=1569527&view=diff
==============================================================================
--- hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/jdbc/TestJdbcDriver.java (original)
+++ hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/jdbc/TestJdbcDriver.java Tue Feb 18 21:07:36 2014
@@ -19,7 +19,7 @@
 package org.apache.hadoop.hive.jdbc;
 
 import static org.apache.hadoop.hive.ql.exec.ExplainTask.EXPL_COLUMN_NAME;
-import static org.apache.hive.common.util.SystemVariables.SET_COLUMN_NAME;
+import static org.apache.hadoop.hive.ql.processors.SetProcessor.SET_COLUMN_NAME;
 
 import java.sql.Connection;
 import java.sql.DatabaseMetaData;

Modified: hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java?rev=1569527&r1=1569526&r2=1569527&view=diff
==============================================================================
--- hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java (original)
+++ hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/jdbc/TestJdbcDriver2.java Tue Feb 18 21:07:36 2014
@@ -19,7 +19,7 @@
 package org.apache.hive.jdbc;
 
 import static org.apache.hadoop.hive.ql.exec.ExplainTask.EXPL_COLUMN_NAME;
-import static org.apache.hive.common.util.SystemVariables.SET_COLUMN_NAME;
+import static org.apache.hadoop.hive.ql.processors.SetProcessor.SET_COLUMN_NAME;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNotNull;
@@ -1858,7 +1858,7 @@ public class TestJdbcDriver2 {
    */
   @Test
   public void testFetchFirstSetCmds() throws Exception {
-    execFetchFirst("set -v", SET_COLUMN_NAME, false);
+    execFetchFirst("set -v", SetProcessor.SET_COLUMN_NAME, false);
   }
 
   /**

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java?rev=1569527&r1=1569526&r2=1569527&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java Tue Feb 18 21:07:36 2014
@@ -138,7 +138,6 @@ import org.apache.hadoop.hive.ql.plan.Re
 import org.apache.hadoop.hive.ql.plan.RevokeDesc;
 import org.apache.hadoop.hive.ql.plan.RoleDDLDesc;
 import org.apache.hadoop.hive.ql.plan.ShowColumnsDesc;
-import org.apache.hadoop.hive.ql.plan.ShowConfDesc;
 import org.apache.hadoop.hive.ql.plan.ShowCreateTableDesc;
 import org.apache.hadoop.hive.ql.plan.ShowDatabasesDesc;
 import org.apache.hadoop.hive.ql.plan.ShowFunctionsDesc;
@@ -177,6 +176,7 @@ import org.apache.hadoop.hive.shims.Shim
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.ToolRunner;
+import org.apache.thrift.TException;
 import org.stringtemplate.v4.ST;
 
 /**
@@ -399,11 +399,6 @@ public class DDLTask extends Task<DDLWor
         return showCreateTable(db, showCreateTbl);
       }
 
-      ShowConfDesc showConf = work.getShowConfDesc();
-      if (showConf != null) {
-        return showConf(db, showConf);
-      }
-
       RoleDDLDesc roleDDLDesc = work.getRoleDDLDesc();
       if (roleDDLDesc != null) {
         return roleDDL(roleDDLDesc);
@@ -466,38 +461,6 @@ public class DDLTask extends Task<DDLWor
     return 0;
   }
 
-  private int showConf(Hive db, ShowConfDesc showConf) throws Exception {
-    ConfVars conf = HiveConf.getConfVars(showConf.getConfName());
-    if (conf == null) {
-      throw new HiveException("invalid configuration name " + showConf.getConfName());
-    }
-    String description = conf.getDescription();
-    String defaltValue = conf.getDefaultValue();
-    DataOutputStream output = getOutputStream(showConf.getResFile());
-    try {
-      if (description != null) {
-        if (defaltValue != null) {
-          output.write(defaltValue.getBytes());
-        }
-        output.write(separator);
-        output.write(conf.typeString().getBytes());
-        output.write(separator);
-        if (description != null) {
-          output.write(description.replaceAll(" *\n *", " ").getBytes());
-        }
-        output.write(terminator);
-      }
-    } finally {
-      output.close();
-    }
-    return 0;
-  }
-
-  private DataOutputStream getOutputStream(Path outputFile) throws Exception {
-    FileSystem fs = outputFile.getFileSystem(conf);
-    return fs.create(outputFile);
-  }
-
   /**
    * First, make sure the source table/partition is not
    * archived/indexes/non-rcfile. If either of these is true, throw an

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java?rev=1569527&r1=1569526&r2=1569527&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java Tue Feb 18 21:07:36 2014
@@ -72,6 +72,7 @@ import java.util.Set;
 import java.util.UUID;
 import java.util.zip.Deflater;
 import java.util.zip.DeflaterOutputStream;
+import java.util.zip.Inflater;
 import java.util.zip.InflaterInputStream;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ExecutionException;

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFile.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFile.java?rev=1569527&r1=1569526&r2=1569527&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFile.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/io/orc/OrcFile.java Tue Feb 18 21:07:36 2014
@@ -27,8 +27,6 @@ import org.apache.hadoop.hive.conf.HiveC
 import org.apache.hadoop.hive.ql.io.orc.Reader.FileMetaInfo;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 
-import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.*;
-
 /**
  * Contains factory methods to read or write ORC files.
  */
@@ -144,12 +142,26 @@ public final class OrcFile {
     WriterOptions(Configuration conf) {
       configuration = conf;
       memoryManagerValue = getMemoryManager(conf);
-      stripeSizeValue = HiveConf.getLongVar(conf, HIVE_ORC_DEFAULT_STRIPE_SIZE);
-      rowIndexStrideValue = HiveConf.getIntVar(conf, HIVE_ORC_DEFAULT_ROW_INDEX_STRIDE);
-      bufferSizeValue = HiveConf.getIntVar(conf, HIVE_ORC_DEFAULT_BUFFER_SIZE);
-      blockPaddingValue = HiveConf.getBoolVar(conf, HIVE_ORC_DEFAULT_BLOCK_PADDING);
-      compressValue = CompressionKind.valueOf(HiveConf.getVar(conf, HIVE_ORC_DEFAULT_COMPRESS));
-      String versionName = HiveConf.getVar(conf, HIVE_ORC_WRITE_FORMAT);
+      stripeSizeValue =
+          conf.getLong(HiveConf.ConfVars.HIVE_ORC_DEFAULT_STRIPE_SIZE.varname,
+              HiveConf.ConfVars.HIVE_ORC_DEFAULT_STRIPE_SIZE.defaultLongVal);
+      rowIndexStrideValue =
+          conf.getInt(HiveConf.ConfVars.HIVE_ORC_DEFAULT_ROW_INDEX_STRIDE
+              .varname, HiveConf.ConfVars.HIVE_ORC_DEFAULT_ROW_INDEX_STRIDE.defaultIntVal);
+      bufferSizeValue =
+          conf.getInt(HiveConf.ConfVars.HIVE_ORC_DEFAULT_BUFFER_SIZE.varname,
+              HiveConf.ConfVars.HIVE_ORC_DEFAULT_BUFFER_SIZE.defaultIntVal);
+      blockPaddingValue =
+          conf.getBoolean(HiveConf.ConfVars.HIVE_ORC_DEFAULT_BLOCK_PADDING
+              .varname, HiveConf.ConfVars.HIVE_ORC_DEFAULT_BLOCK_PADDING
+              .defaultBoolVal);
+      compressValue = 
+          CompressionKind.valueOf(conf.get(HiveConf.ConfVars
+              .HIVE_ORC_DEFAULT_COMPRESS.varname,
+              HiveConf.ConfVars
+              .HIVE_ORC_DEFAULT_COMPRESS.defaultVal));
+      String versionName =
+        conf.get(HiveConf.ConfVars.HIVE_ORC_WRITE_FORMAT.varname);
       if (versionName == null) {
         versionValue = Version.CURRENT;
       } else {

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java?rev=1569527&r1=1569526&r2=1569527&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java Tue Feb 18 21:07:36 2014
@@ -29,6 +29,7 @@ import org.apache.hadoop.hive.ql.securit
 import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider;
 import org.apache.hadoop.hive.ql.security.authorization.DefaultHiveAuthorizationProvider;
 import org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthorizer;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthorizerFactory;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizerFactory;
 import org.apache.hadoop.io.Text;

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java?rev=1569527&r1=1569526&r2=1569527&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java Tue Feb 18 21:07:36 2014
@@ -109,7 +109,6 @@ import org.apache.hadoop.hive.ql.plan.Pl
 import org.apache.hadoop.hive.ql.plan.RenamePartitionDesc;
 import org.apache.hadoop.hive.ql.plan.RoleDDLDesc;
 import org.apache.hadoop.hive.ql.plan.ShowColumnsDesc;
-import org.apache.hadoop.hive.ql.plan.ShowConfDesc;
 import org.apache.hadoop.hive.ql.plan.ShowCreateTableDesc;
 import org.apache.hadoop.hive.ql.plan.ShowDatabasesDesc;
 import org.apache.hadoop.hive.ql.plan.ShowFunctionsDesc;
@@ -311,10 +310,6 @@ public class DDLSemanticAnalyzer extends
       ctx.setResFile(ctx.getLocalTmpPath());
       analyzeShowDbLocks(ast);
       break;
-    case HiveParser.TOK_SHOWCONF:
-      ctx.setResFile(ctx.getLocalTmpPath());
-      analyzeShowConf(ast);
-      break;
     case HiveParser.TOK_DESCFUNCTION:
       ctx.setResFile(ctx.getLocalTmpPath());
       analyzeDescFunction(ast);
@@ -2195,14 +2190,6 @@ public class DDLSemanticAnalyzer extends
     ctx.setNeedLockMgr(true);
   }
 
-  private void analyzeShowConf(ASTNode ast) throws SemanticException {
-    String confName = stripQuotes(ast.getChild(0).getText());
-    ShowConfDesc showConfDesc = new ShowConfDesc(ctx.getResFile(), confName);
-    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
-        showConfDesc), conf));
-    setFetchTask(createFetchTask(showConfDesc.getSchema()));
-  }
-
   /**
    * Add the task according to the parsed command tree. This is used for the CLI
    * command "LOCK TABLE ..;".

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g?rev=1569527&r1=1569526&r2=1569527&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g Tue Feb 18 21:07:36 2014
@@ -285,7 +285,6 @@ KW_ROLES: 'ROLES';
 KW_INNER: 'INNER';
 KW_EXCHANGE: 'EXCHANGE';
 KW_ADMIN: 'ADMIN';
-KW_CONF: 'CONF';
 
 // Operators
 // NOTE: if you add a new function/operator, add it to sysFuncNames so that describe function _FUNC_ will work.

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g?rev=1569527&r1=1569526&r2=1569527&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g Tue Feb 18 21:07:36 2014
@@ -163,7 +163,6 @@ TOK_SHOW_CREATETABLE;
 TOK_SHOW_TABLESTATUS;
 TOK_SHOW_TBLPROPERTIES;
 TOK_SHOWLOCKS;
-TOK_SHOWCONF;
 TOK_LOCKTABLE;
 TOK_UNLOCKTABLE;
 TOK_LOCKDB;
@@ -1291,8 +1290,6 @@ showStatement
     | KW_SHOW KW_LOCKS KW_DATABASE (dbName=Identifier) (isExtended=KW_EXTENDED)? -> ^(TOK_SHOWDBLOCKS $dbName $isExtended?)
     | KW_SHOW (showOptions=KW_FORMATTED)? (KW_INDEX|KW_INDEXES) KW_ON showStmtIdentifier ((KW_FROM|KW_IN) db_name=identifier)?
     -> ^(TOK_SHOWINDEXES showStmtIdentifier $showOptions? $db_name?)
-    | KW_SHOW KW_CONF StringLiteral
-    -> ^(TOK_SHOWCONF StringLiteral)
     ;
 
 lockStatement

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java?rev=1569527&r1=1569526&r2=1569527&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java Tue Feb 18 21:07:36 2014
@@ -69,7 +69,6 @@ public final class SemanticAnalyzerFacto
     commandType.put(HiveParser.TOK_SHOWPARTITIONS, HiveOperation.SHOWPARTITIONS);
     commandType.put(HiveParser.TOK_SHOWLOCKS, HiveOperation.SHOWLOCKS);
     commandType.put(HiveParser.TOK_SHOWDBLOCKS, HiveOperation.SHOWLOCKS);
-    commandType.put(HiveParser.TOK_SHOWCONF, HiveOperation.SHOWCONF);
     commandType.put(HiveParser.TOK_CREATEFUNCTION, HiveOperation.CREATEFUNCTION);
     commandType.put(HiveParser.TOK_DROPFUNCTION, HiveOperation.DROPFUNCTION);
     commandType.put(HiveParser.TOK_CREATEMACRO, HiveOperation.CREATEMACRO);
@@ -194,7 +193,6 @@ public final class SemanticAnalyzerFacto
       case HiveParser.TOK_SHOWINDEXES:
       case HiveParser.TOK_SHOWLOCKS:
       case HiveParser.TOK_SHOWDBLOCKS:
-      case HiveParser.TOK_SHOWCONF:
       case HiveParser.TOK_CREATEINDEX:
       case HiveParser.TOK_DROPINDEX:
       case HiveParser.TOK_ALTERTABLE_CLUSTER_SORT:

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/VariableSubstitution.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/VariableSubstitution.java?rev=1569527&r1=1569526&r2=1569527&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/VariableSubstitution.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/parse/VariableSubstitution.java Tue Feb 18 21:07:36 2014
@@ -17,48 +17,79 @@
  */
 package org.apache.hadoop.hive.ql.parse;
 
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.ql.processors.SetProcessor;
 import org.apache.hadoop.hive.ql.session.SessionState;
-import org.apache.hive.common.util.SystemVariables;
-
-import java.util.Map;
 
-public class VariableSubstitution extends SystemVariables {
+public class VariableSubstitution {
 
   private static final Log l4j = LogFactory.getLog(VariableSubstitution.class);
+  protected static Pattern varPat = Pattern.compile("\\$\\{[^\\}\\$\u0020]+\\}");
 
-  @Override
-  protected String getSubstitute(HiveConf conf, String var) {
-    String val = super.getSubstitute(conf, var);
+  private String getSubstitute(HiveConf conf, String var) {
+    String val = null;
+    try {
+      if (var.startsWith(SetProcessor.SYSTEM_PREFIX)) {
+        val = System.getProperty(var.substring(SetProcessor.SYSTEM_PREFIX.length()));
+      }
+    } catch(SecurityException se) {
+      l4j.warn("Unexpected SecurityException in Configuration", se);
+    }
+    if (val ==null){
+      if (var.startsWith(SetProcessor.ENV_PREFIX)){
+        val = System.getenv(var.substring(SetProcessor.ENV_PREFIX.length()));
+      }
+    }
     if (val == null) {
-      if (var.startsWith(HIVECONF_PREFIX)) {
-        val = conf.get(var.substring(HIVECONF_PREFIX.length()));
+      if (var.startsWith(SetProcessor.HIVECONF_PREFIX)){
+        val = conf.get(var.substring(SetProcessor.HIVECONF_PREFIX.length()));
       }
     }
-    if (val == null){
-      Map<String,String> vars = SessionState.get().getHiveVariables();
-      if (var.startsWith(HIVEVAR_PREFIX)) {
-        val =  vars.get(var.substring(HIVEVAR_PREFIX.length()));
+    if (val ==null){
+      if(var.startsWith(SetProcessor.HIVEVAR_PREFIX)){
+        val =  SessionState.get().getHiveVariables().get(var.substring(SetProcessor.HIVEVAR_PREFIX.length()));
       } else {
-        val = vars.get(var);
+        val = SessionState.get().getHiveVariables().get(var);
       }
     }
     return val;
   }
 
-  @Override
   public String substitute (HiveConf conf, String expr) {
-    if (conf.getBoolVar(ConfVars.HIVEVARIABLESUBSTITUTE)) {
-      l4j.debug("Substitution is on: " + expr);
+
+    if (conf.getBoolVar(ConfVars.HIVEVARIABLESUBSTITUTE)){
+      l4j.debug("Substitution is on: "+expr);
     } else {
       return expr;
     }
     if (expr == null) {
       return null;
     }
-    return super.substitute(conf, expr);
+    Matcher match = varPat.matcher("");
+    String eval = expr;
+    for(int s=0;s<conf.getIntVar(ConfVars.HIVEVARIABLESUBSTITUTEDEPTH); s++) {
+      match.reset(eval);
+      if (!match.find()) {
+        return eval;
+      }
+      String var = match.group();
+      var = var.substring(2, var.length()-1); // remove ${ .. }
+      String val = getSubstitute(conf, var);
+
+      if (val == null) {
+        l4j.debug("Interpolation result: "+eval);
+        return eval; // return literal, no substitution found
+      }
+      // substitute
+      eval = eval.substring(0, match.start())+val+eval.substring(match.end());
+    }
+    throw new IllegalStateException("Variable substitution depth too large: "
+                                    + conf.getIntVar(ConfVars.HIVEVARIABLESUBSTITUTEDEPTH) + " " + expr);
   }
 }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java?rev=1569527&r1=1569526&r2=1569527&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java Tue Feb 18 21:07:36 2014
@@ -74,8 +74,6 @@ public class DDLWork implements Serializ
   private RevokeDesc revokeDesc;
   private GrantRevokeRoleDDL grantRevokeRoleDDL;
 
-  private ShowConfDesc showConfDesc;
-
   boolean needLock = false;
 
   /**
@@ -139,12 +137,6 @@ public class DDLWork implements Serializ
     this.truncateTblDesc = truncateTblDesc;
   }
 
-  public DDLWork(HashSet<ReadEntity> inputs, HashSet<WriteEntity> outputs,
-      ShowConfDesc showConfDesc) {
-    this(inputs, outputs);
-    this.showConfDesc = showConfDesc;
-  }
-
   public DescDatabaseDesc getDescDatabaseDesc() {
     return descDbDesc;
   }
@@ -1093,12 +1085,4 @@ public class DDLWork implements Serializ
       AlterTableExchangePartition alterTableExchangePartition) {
     this.alterTableExchangePartition = alterTableExchangePartition;
   }
-
-  public ShowConfDesc getShowConfDesc() {
-    return showConfDesc;
-  }
-
-  public void setShowConfDesc(ShowConfDesc showConfDesc) {
-    this.showConfDesc = showConfDesc;
-  }
 }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java?rev=1569527&r1=1569526&r2=1569527&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java Tue Feb 18 21:07:36 2014
@@ -66,7 +66,6 @@ public enum HiveOperation {
   SHOWINDEXES("SHOWINDEXES", null, null),
   SHOWPARTITIONS("SHOWPARTITIONS", null, null),
   SHOWLOCKS("SHOWLOCKS", null, null),
-  SHOWCONF("SHOWCONF", null, null),
   CREATEFUNCTION("CREATEFUNCTION", null, null),
   DROPFUNCTION("DROPFUNCTION", null, null),
   CREATEMACRO("CREATEMACRO", null, null),

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java?rev=1569527&r1=1569526&r2=1569527&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java Tue Feb 18 21:07:36 2014
@@ -22,8 +22,6 @@ import static org.apache.hadoop.hive.ser
 import static org.apache.hadoop.hive.serde.serdeConstants.STRING_TYPE_NAME;
 import static org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe.defaultNullString;
 
-import static org.apache.hive.common.util.SystemVariables.*;
-
 import java.util.Map;
 import java.util.Properties;
 import java.util.SortedMap;
@@ -41,7 +39,12 @@ import org.apache.hadoop.hive.ql.session
  */
 public class SetProcessor implements CommandProcessor {
 
-  private static final String prefix = "set: ";
+  private static String prefix = "set: ";
+  public static final String ENV_PREFIX = "env:";
+  public static final String SYSTEM_PREFIX = "system:";
+  public static final String HIVECONF_PREFIX = "hiveconf:";
+  public static final String HIVEVAR_PREFIX = "hivevar:";
+  public static final String SET_COLUMN_NAME = "set";
 
   public static boolean getBoolean(String value) {
     if (value.equals("on") || value.equals("true")) {
@@ -66,7 +69,7 @@ public class SetProcessor implements Com
 
     // Inserting hive variables
     for (String s : ss.getHiveVariables().keySet()) {
-      sortedMap.put(HIVEVAR_PREFIX + s, ss.getHiveVariables().get(s));
+      sortedMap.put(SetProcessor.HIVEVAR_PREFIX + s, ss.getHiveVariables().get(s));
     }
 
     for (Map.Entry<String, String> entries : sortedMap.entrySet()) {
@@ -105,23 +108,23 @@ public class SetProcessor implements Com
     if (varvalue.contains("\n")){
       ss.err.println("Warning: Value had a \\n character in it.");
     }
-    if (varname.startsWith(ENV_PREFIX)){
+    if (varname.startsWith(SetProcessor.ENV_PREFIX)){
       ss.err.println("env:* variables can not be set.");
       return new CommandProcessorResponse(1);
-    } else if (varname.startsWith(SYSTEM_PREFIX)){
-      String propName = varname.substring(SYSTEM_PREFIX.length());
+    } else if (varname.startsWith(SetProcessor.SYSTEM_PREFIX)){
+      String propName = varname.substring(SetProcessor.SYSTEM_PREFIX.length());
       System.getProperties().setProperty(propName, new VariableSubstitution().substitute(ss.getConf(),varvalue));
       return new CommandProcessorResponse(0);
-    } else if (varname.startsWith(HIVECONF_PREFIX)){
-      String propName = varname.substring(HIVECONF_PREFIX.length());
+    } else if (varname.startsWith(SetProcessor.HIVECONF_PREFIX)){
+      String propName = varname.substring(SetProcessor.HIVECONF_PREFIX.length());
       try {
         setConf(varname, propName, varvalue, false);
         return new CommandProcessorResponse(0);
       } catch (IllegalArgumentException e) {
         return new CommandProcessorResponse(1, e.getMessage(), "42000");
       }
-    } else if (varname.startsWith(HIVEVAR_PREFIX)) {
-      String propName = varname.substring(HIVEVAR_PREFIX.length());
+    } else if (varname.startsWith(SetProcessor.HIVEVAR_PREFIX)) {
+      String propName = varname.substring(SetProcessor.HIVEVAR_PREFIX.length());
       ss.getHiveVariables().put(propName, new VariableSubstitution().substitute(ss.getConf(),varvalue));
       return new CommandProcessorResponse(0);
     } else {
@@ -166,7 +169,7 @@ public class SetProcessor implements Com
 
   private SortedMap<String,String> propertiesToSortedMap(Properties p){
     SortedMap<String,String> sortedPropMap = new TreeMap<String,String>();
-    for (Map.Entry<Object, Object> entry : p.entrySet() ){
+    for (Map.Entry<Object, Object> entry :System.getProperties().entrySet() ){
       sortedPropMap.put( (String) entry.getKey(), (String) entry.getValue());
     }
     return sortedPropMap;
@@ -185,38 +188,38 @@ public class SetProcessor implements Com
       ss.out.println("silent" + "=" + ss.getIsSilent());
       return createProcessorSuccessResponse();
     }
-    if (varname.startsWith(SYSTEM_PREFIX)) {
-      String propName = varname.substring(SYSTEM_PREFIX.length());
+    if (varname.startsWith(SetProcessor.SYSTEM_PREFIX)){
+      String propName = varname.substring(SetProcessor.SYSTEM_PREFIX.length());
       String result = System.getProperty(propName);
-      if (result != null) {
-        ss.out.println(SYSTEM_PREFIX + propName + "=" + result);
+      if (result != null){
+        ss.out.println(SetProcessor.SYSTEM_PREFIX+propName + "=" + result);
         return createProcessorSuccessResponse();
       } else {
-        ss.out.println(propName + " is undefined as a system property");
+        ss.out.println( propName + " is undefined as a system property");
         return new CommandProcessorResponse(1);
       }
-    } else if (varname.indexOf(ENV_PREFIX) == 0) {
+    } else if (varname.indexOf(SetProcessor.ENV_PREFIX)==0){
       String var = varname.substring(ENV_PREFIX.length());
-      if (System.getenv(var) != null) {
-        ss.out.println(ENV_PREFIX + var + "=" + System.getenv(var));
+      if (System.getenv(var)!=null){
+        ss.out.println(SetProcessor.ENV_PREFIX+var + "=" + System.getenv(var));
         return createProcessorSuccessResponse();
       } else {
         ss.out.println(varname + " is undefined as an environmental variable");
         return new CommandProcessorResponse(1);
       }
-    } else if (varname.indexOf(HIVECONF_PREFIX) == 0) {
-      String var = varname.substring(HIVECONF_PREFIX.length());
-      if (ss.getConf().get(var) != null) {
-        ss.out.println(HIVECONF_PREFIX + var + "=" + ss.getConf().get(var));
+    } else if (varname.indexOf(SetProcessor.HIVECONF_PREFIX)==0) {
+      String var = varname.substring(SetProcessor.HIVECONF_PREFIX.length());
+      if (ss.getConf().get(var)!=null){
+        ss.out.println(SetProcessor.HIVECONF_PREFIX+var + "=" + ss.getConf().get(var));
         return createProcessorSuccessResponse();
       } else {
         ss.out.println(varname + " is undefined as a hive configuration variable");
         return new CommandProcessorResponse(1);
       }
-    } else if (varname.indexOf(HIVEVAR_PREFIX) == 0) {
-      String var = varname.substring(HIVEVAR_PREFIX.length());
-      if (ss.getHiveVariables().get(var) != null) {
-        ss.out.println(HIVEVAR_PREFIX + var + "=" + ss.getHiveVariables().get(var));
+    } else if (varname.indexOf(SetProcessor.HIVEVAR_PREFIX)==0) {
+      String var = varname.substring(SetProcessor.HIVEVAR_PREFIX.length());
+      if (ss.getHiveVariables().get(var)!=null){
+        ss.out.println(SetProcessor.HIVEVAR_PREFIX+var + "=" + ss.getHiveVariables().get(var));
         return createProcessorSuccessResponse();
       } else {
         ss.out.println(varname + " is undefined as a hive variable");

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java?rev=1569527&r1=1569526&r2=1569527&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java Tue Feb 18 21:07:36 2014
@@ -69,7 +69,6 @@ public enum HiveOperationType {
   SHOWINDEXES,
   SHOWPARTITIONS,
   SHOWLOCKS,
-  SHOWCONF,
   CREATEFUNCTION,
   DROPFUNCTION,
   CREATEMACRO,

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java?rev=1569527&r1=1569526&r2=1569527&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java Tue Feb 18 21:07:36 2014
@@ -160,7 +160,6 @@ public class Operation2Privilege {
     op2Priv.put(HiveOperationType.SHOWINDEXES, new InOutPrivs(null, null));
     op2Priv.put(HiveOperationType.SHOWPARTITIONS, new InOutPrivs(null, null));
     op2Priv.put(HiveOperationType.SHOWLOCKS, new InOutPrivs(null, null));
-    op2Priv.put(HiveOperationType.SHOWCONF, new InOutPrivs(null, null));
     op2Priv.put(HiveOperationType.CREATEFUNCTION, new InOutPrivs(null, null));
     op2Priv.put(HiveOperationType.DROPFUNCTION, new InOutPrivs(null, null));
     op2Priv.put(HiveOperationType.CREATEMACRO, new InOutPrivs(null, null));