You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ec...@apache.org on 2013/10/06 03:08:26 UTC

svn commit: r1529558 - in /hive/trunk: cli/src/java/org/apache/hadoop/hive/cli/ common/src/java/org/apache/hadoop/hive/conf/ conf/ hwi/src/java/org/apache/hadoop/hive/hwi/ ql/src/java/org/apache/hadoop/hive/ql/processors/ ql/src/test/org/apache/hadoop/...

Author: ecapriolo
Date: Sun Oct  6 01:08:25 2013
New Revision: 1529558

URL: http://svn.apache.org/r1529558
Log:
HIVE-5250 Allow admins to enable and disable non query commands (Brock Noland via egc)


Added:
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/processors/HiveCommand.java
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/processors/
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/processors/TestCommandProcessorFactory.java
Modified:
    hive/trunk/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java
    hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
    hive/trunk/conf/hive-default.xml.template
    hive/trunk/hwi/src/java/org/apache/hadoop/hive/hwi/HWISessionItem.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessorFactory.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/operation/AddResourceOperation.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/operation/DeleteResourceOperation.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/operation/DfsOperation.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/operation/OperationManager.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/operation/SetOperation.java

Modified: hive/trunk/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java
URL: http://svn.apache.org/viewvc/hive/trunk/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java?rev=1529558&r1=1529557&r2=1529558&view=diff
==============================================================================
--- hive/trunk/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java (original)
+++ hive/trunk/cli/src/java/org/apache/hadoop/hive/cli/CliDriver.java Sun Oct  6 01:08:25 2013
@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.hive.cli;
 
+import static org.apache.hadoop.util.StringUtils.stringifyException;
+
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.FileNotFoundException;
@@ -30,6 +32,7 @@ import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.sql.SQLException;
 
 import jline.ArgumentCompletor;
 import jline.ArgumentCompletor.AbstractArgumentDelimiter;
@@ -122,7 +125,7 @@ public class CliDriver {
           this.processFile(cmd_1);
         } catch (IOException e) {
           console.printError("Failed processing file "+ cmd_1 +" "+ e.getLocalizedMessage(),
-            org.apache.hadoop.util.StringUtils.stringifyException(e));
+            stringifyException(e));
           ret = 1;
         }
       }
@@ -146,7 +149,7 @@ public class CliDriver {
         }
       } catch (Exception e) {
         console.printError("Exception raised from Shell command " + e.getLocalizedMessage(),
-            org.apache.hadoop.util.StringUtils.stringifyException(e));
+            stringifyException(e));
         ret = 1;
       }
 
@@ -212,8 +215,14 @@ public class CliDriver {
         }
       }
     } else { // local mode
-      CommandProcessor proc = CommandProcessorFactory.get(tokens[0], (HiveConf) conf);
-      ret = processLocalCmd(cmd, proc, ss);
+      try {
+        CommandProcessor proc = CommandProcessorFactory.get(tokens[0], (HiveConf) conf);
+        ret = processLocalCmd(cmd, proc, ss);
+      } catch (SQLException e) {
+        console.printError("Failed processing command " + tokens[0] + " " + e.getLocalizedMessage(),
+          org.apache.hadoop.util.StringUtils.stringifyException(e));
+        ret = 1;
+      }
     }
 
     return ret;

Modified: hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
URL: http://svn.apache.org/viewvc/hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java?rev=1529558&r1=1529557&r2=1529558&view=diff
==============================================================================
--- hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (original)
+++ hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java Sun Oct  6 01:08:25 2013
@@ -774,7 +774,9 @@ public class HiveConf extends Configurat
     HIVE_SERVER2_TABLE_TYPE_MAPPING("hive.server2.table.type.mapping", "CLASSIC"),
     HIVE_SERVER2_SESSION_HOOK("hive.server2.session.hook", ""),
 
-    HIVE_CONF_RESTRICTED_LIST("hive.conf.restricted.list", null),
+    HIVE_SECURITY_COMMAND_WHITELIST("hive.security.command.whitelist", "set,reset,dfs,add,delete"),
+
+    HIVE_CONF_RESTRICTED_LIST("hive.conf.restricted.list", ""),
 
     // If this is set all move tasks at the end of a multi-insert query will only begin once all
     // outputs are ready
@@ -1165,9 +1167,10 @@ public class HiveConf extends Configurat
     }
 
     // setup list of conf vars that are not allowed to change runtime
-    String restrictListStr = this.get(ConfVars.HIVE_CONF_RESTRICTED_LIST.toString());
-    if (restrictListStr != null) {
-      for (String entry : restrictListStr.split(",")) {
+    String restrictListStr = this.get(ConfVars.HIVE_CONF_RESTRICTED_LIST.toString(), "").trim();
+    for (String entry : restrictListStr.split(",")) {
+      entry = entry.trim();
+      if (!entry.isEmpty()) {
         restrictList.add(entry);
       }
     }

Modified: hive/trunk/conf/hive-default.xml.template
URL: http://svn.apache.org/viewvc/hive/trunk/conf/hive-default.xml.template?rev=1529558&r1=1529557&r2=1529558&view=diff
==============================================================================
--- hive/trunk/conf/hive-default.xml.template (original)
+++ hive/trunk/conf/hive-default.xml.template Sun Oct  6 01:08:25 2013
@@ -1514,6 +1514,18 @@
 </property>
 
 <property>
+  <name>hive.security.command.whitelist</name>
+  <value>set,reset,dfs,add,delete</value>
+  <description>Comma seperated list of non-SQL Hive commands users are authorized to execute</description>
+</property>
+
+<property>
+  <name>hive.conf.restricted.list</name>
+  <value></value>
+  <description>Comma seperated list of configuration options which are immutable at runtime</description>
+</property>
+
+<property>
   <name>hive.metastore.authorization.storage.checks</name>
   <value>false</value>
   <description>Should the metastore do authorization checks against the underlying storage

Modified: hive/trunk/hwi/src/java/org/apache/hadoop/hive/hwi/HWISessionItem.java
URL: http://svn.apache.org/viewvc/hive/trunk/hwi/src/java/org/apache/hadoop/hive/hwi/HWISessionItem.java?rev=1529558&r1=1529557&r2=1529558&view=diff
==============================================================================
--- hive/trunk/hwi/src/java/org/apache/hadoop/hive/hwi/HWISessionItem.java (original)
+++ hive/trunk/hwi/src/java/org/apache/hadoop/hive/hwi/HWISessionItem.java Sun Oct  6 01:08:25 2013
@@ -24,6 +24,7 @@ import java.io.IOException;
 import java.io.PrintStream;
 import java.util.ArrayList;
 import java.util.List;
+import java.sql.SQLException;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -332,8 +333,12 @@ public class HWISessionItem implements R
       String cmd_trimmed = cmd.trim();
       String[] tokens = cmd_trimmed.split("\\s+");
       String cmd_1 = cmd_trimmed.substring(tokens[0].length()).trim();
-
-      CommandProcessor proc = CommandProcessorFactory.get(tokens[0]);
+      CommandProcessor proc = null;
+      try {
+        proc = CommandProcessorFactory.get(tokens[0]);
+      } catch (SQLException e) {
+        l4j.error(getSessionName() + " error processing " + cmd, e);
+      }
       if (proc != null) {
         if (proc instanceof Driver) {
           Driver qp = (Driver) proc;

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessorFactory.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessorFactory.java?rev=1529558&r1=1529557&r2=1529558&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessorFactory.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessorFactory.java Sun Oct  6 01:08:25 2013
@@ -19,8 +19,12 @@
 package org.apache.hadoop.hive.ql.processors;
 
 import static org.apache.commons.lang.StringUtils.isBlank;
-import java.util.Map;
+import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.sql.SQLException;
 
 import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.ql.session.SessionState;
@@ -36,30 +40,58 @@ public final class CommandProcessorFacto
     // prevent instantiation
   }
 
-  static Map<HiveConf, Driver> mapDrivers = new HashMap<HiveConf, Driver>();
-  public static CommandProcessor get(String cmd) {
+  private static final Map<HiveConf, Driver> mapDrivers = Collections.synchronizedMap(new HashMap<HiveConf, Driver>());
+
+  public static CommandProcessor get(String cmd)
+      throws SQLException {
     return get(cmd, null);
   }
 
-  public static CommandProcessor get(String cmd, HiveConf conf) {
-    String cmdl = cmd.toLowerCase();
+  public static CommandProcessor getForHiveCommand(String cmd, HiveConf conf)
+      throws SQLException {
+    HiveCommand hiveCommand = HiveCommand.find(cmd);
+    if (hiveCommand == null || isBlank(cmd)) {
+      return null;
+    }
+    if (conf == null) {
+      conf = new HiveConf();
+    }
+    Set<String> availableCommands = new HashSet<String>();
+    for (String availableCommand : conf.getVar(HiveConf.ConfVars.HIVE_SECURITY_COMMAND_WHITELIST).split(",")) {
+      availableCommands.add(availableCommand.toLowerCase().trim());
+    }
+    if (!availableCommands.contains(cmd.trim().toLowerCase())) {
+      throw new SQLException("Insufficient privileges to execute " + cmd, "42000");
+    }
+    switch (hiveCommand) {
+      case SET:
+        return new SetProcessor();
+      case RESET:
+        return new ResetProcessor();
+      case DFS:
+        SessionState ss = SessionState.get();
+        return new DfsProcessor(ss.getConf());
+      case ADD:
+        return new AddResourceProcessor();
+      case DELETE:
+        return new DeleteResourceProcessor();
+      default:
+        throw new AssertionError("Unknown HiveCommand " + hiveCommand);
+    }
+  }
 
-    if ("set".equals(cmdl)) {
-      return new SetProcessor();
-    } else if ("reset".equals(cmdl)) {
-      return new ResetProcessor();
-    } else if ("dfs".equals(cmdl)) {
-      SessionState ss = SessionState.get();
-      return new DfsProcessor(ss.getConf());
-    } else if ("add".equals(cmdl)) {
-      return new AddResourceProcessor();
-    } else if ("delete".equals(cmdl)) {
-      return new DeleteResourceProcessor();
-    } else if (!isBlank(cmd)) {
+  public static CommandProcessor get(String cmd, HiveConf conf)
+      throws SQLException {
+    CommandProcessor result = getForHiveCommand(cmd, conf);
+    if (result != null) {
+      return result;
+    }
+    if (isBlank(cmd)) {
+      return null;
+    } else {
       if (conf == null) {
         return new Driver();
       }
-
       Driver drv = mapDrivers.get(conf);
       if (drv == null) {
         drv = new Driver();
@@ -68,8 +100,6 @@ public final class CommandProcessorFacto
       drv.init();
       return drv;
     }
-
-    return null;
   }
 
   public static void clean(HiveConf conf) {

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/processors/HiveCommand.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/processors/HiveCommand.java?rev=1529558&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/processors/HiveCommand.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/processors/HiveCommand.java Sun Oct  6 01:08:25 2013
@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.processors;
+
+import java.util.HashSet;
+import java.util.Set;
+
+/*
+ * HiveCommand is non-SQL statement such as setting a property or
+ * adding a resource.
+ **/
+public enum HiveCommand {
+  SET(),
+  RESET(),
+  DFS(),
+  ADD(),
+  DELETE();
+  private static final Set<String> COMMANDS = new HashSet<String>();
+  static {
+    for (HiveCommand command : HiveCommand.values()) {
+      COMMANDS.add(command.name());
+    }
+  }
+  public static HiveCommand find(String command) {
+    if (command != null) {
+      command = command.trim().toUpperCase();
+      if (COMMANDS.contains(command)) {
+        return HiveCommand.valueOf(command);
+      }
+    }
+    return null;
+  }
+}

Added: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/processors/TestCommandProcessorFactory.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/processors/TestCommandProcessorFactory.java?rev=1529558&view=auto
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/processors/TestCommandProcessorFactory.java (added)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/processors/TestCommandProcessorFactory.java Sun Oct  6 01:08:25 2013
@@ -0,0 +1,68 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.processors;
+
+import java.sql.SQLException;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestCommandProcessorFactory {
+
+  private HiveConf conf;
+
+  @Before
+  public void setUp() throws Exception {
+    conf = new HiveConf();
+  }
+
+  @Test
+  public void testInvalidCommands() throws Exception {
+    Assert.assertNull("Null should have returned null", CommandProcessorFactory.getForHiveCommand(null, conf));
+    Assert.assertNull("Blank should have returned null", CommandProcessorFactory.getForHiveCommand(" ", conf));
+    Assert.assertNull("SQL should have returned null", CommandProcessorFactory.getForHiveCommand("SELECT * FROM TABLE", conf));
+  }
+  @Test
+  public void testAvailableCommands() throws Exception {
+    SessionState.start(conf);
+    for (HiveCommand command : HiveCommand.values()) {
+      String cmd = command.name();
+      Assert.assertNotNull("Cmd " + cmd + " not return null", CommandProcessorFactory.getForHiveCommand(cmd, conf));
+    }
+    for (HiveCommand command : HiveCommand.values()) {
+      String cmd = command.name().toLowerCase();
+      Assert.assertNotNull("Cmd " + cmd + " not return null", CommandProcessorFactory.getForHiveCommand(cmd, conf));
+    }
+    conf.set(HiveConf.ConfVars.HIVE_SECURITY_COMMAND_WHITELIST.toString(), "");
+    for (HiveCommand command : HiveCommand.values()) {
+      String cmd = command.name();
+      try {
+        CommandProcessorFactory.getForHiveCommand(cmd, conf);
+        Assert.fail("Expected SQLException for " + cmd + " as available commands is empty");
+      } catch (SQLException e) {
+        Assert.assertEquals("Insufficient privileges to execute " + cmd, e.getMessage());
+        Assert.assertEquals("42000", e.getSQLState());
+      }
+    }
+  }
+}

Modified: hive/trunk/service/src/java/org/apache/hive/service/cli/operation/AddResourceOperation.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/operation/AddResourceOperation.java?rev=1529558&r1=1529557&r2=1529558&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/operation/AddResourceOperation.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/operation/AddResourceOperation.java Sun Oct  6 01:08:25 2013
@@ -1,38 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hive.service.cli.operation;
-
-import java.util.Map;
-
-import org.apache.hadoop.hive.ql.processors.AddResourceProcessor;
-import org.apache.hive.service.cli.session.HiveSession;
-
-/**
- * HiveAddResourceOperation.
- *
- */
-public class AddResourceOperation extends HiveCommandOperation {
-
-  protected AddResourceOperation(HiveSession parentSession, String statement,
-      Map<String, String> confOverlay) {
-    super(parentSession, statement, confOverlay);
-    setCommandProcessor(new AddResourceProcessor());
-  }
-
-}

Modified: hive/trunk/service/src/java/org/apache/hive/service/cli/operation/DeleteResourceOperation.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/operation/DeleteResourceOperation.java?rev=1529558&r1=1529557&r2=1529558&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/operation/DeleteResourceOperation.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/operation/DeleteResourceOperation.java Sun Oct  6 01:08:25 2013
@@ -1,38 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hive.service.cli.operation;
-
-import java.util.Map;
-
-import org.apache.hadoop.hive.ql.processors.DeleteResourceProcessor;
-import org.apache.hive.service.cli.session.HiveSession;
-
-/**
- * HiveDeleteResourceOperation.
- *
- */
-public class DeleteResourceOperation extends HiveCommandOperation {
-
-  protected DeleteResourceOperation(HiveSession parentSession, String statement,
-      Map<String, String> confOverlay) {
-    super(parentSession, statement, confOverlay);
-    setCommandProcessor(new DeleteResourceProcessor());
-  }
-
-}

Modified: hive/trunk/service/src/java/org/apache/hive/service/cli/operation/DfsOperation.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/operation/DfsOperation.java?rev=1529558&r1=1529557&r2=1529558&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/operation/DfsOperation.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/operation/DfsOperation.java Sun Oct  6 01:08:25 2013
@@ -1,38 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hive.service.cli.operation;
-
-import java.util.Map;
-
-import org.apache.hadoop.hive.ql.processors.DfsProcessor;
-import org.apache.hive.service.cli.session.HiveSession;
-
-/**
- * HiveDfsCommandOperation.
- *
- */
-public class DfsOperation extends HiveCommandOperation {
-
-  protected DfsOperation(HiveSession parentSession, String statement,
-      Map<String, String> confOverlay) {
-    super(parentSession, statement, confOverlay);
-    setCommandProcessor(new DfsProcessor(parentSession.getHiveConf()));
-  }
-
-}

Modified: hive/trunk/service/src/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java?rev=1529558&r1=1529557&r2=1529558&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java Sun Oct  6 01:08:25 2013
@@ -20,8 +20,16 @@ package org.apache.hive.service.cli.oper
 
 
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Map;
+import java.util.Set;
+import java.sql.SQLException;
 
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory;
+import org.apache.hadoop.hive.ql.processors.CommandProcessor;
+import org.apache.hadoop.hive.ql.processors.HiveCommand;
+import org.apache.hive.service.cli.HiveSQLException;
 import org.apache.hive.service.cli.OperationType;
 import org.apache.hive.service.cli.session.HiveSession;
 
@@ -40,20 +48,19 @@ public abstract class ExecuteStatementOp
   }
 
   public static ExecuteStatementOperation newExecuteStatementOperation(
-      HiveSession parentSession, String statement, Map<String, String> confOverlay, boolean runAsync) {
+      HiveSession parentSession, String statement, Map<String, String> confOverlay, boolean runAsync)
+      throws HiveSQLException {
     String[] tokens = statement.trim().split("\\s+");
     String command = tokens[0].toLowerCase();
-
-    if ("set".equals(command)) {
-      return new SetOperation(parentSession, statement, confOverlay);
-    } else if ("dfs".equals(command)) {
-      return new DfsOperation(parentSession, statement, confOverlay);
-    } else if ("add".equals(command)) {
-      return new AddResourceOperation(parentSession, statement, confOverlay);
-    } else if ("delete".equals(command)) {
-      return new DeleteResourceOperation(parentSession, statement, confOverlay);
-    } else {
+    CommandProcessor processor = null;
+    try {
+      processor = CommandProcessorFactory.getForHiveCommand(tokens[0], parentSession.getHiveConf());
+    } catch (SQLException e) {
+      throw new HiveSQLException(e.getMessage(), e.getSQLState(), e);
+    }
+    if (processor == null) {
       return new SQLOperation(parentSession, statement, confOverlay, runAsync);
     }
+    return new HiveCommandOperation(parentSession, statement, processor, confOverlay);
   }
 }

Modified: hive/trunk/service/src/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java?rev=1529558&r1=1529557&r2=1529558&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java Sun Oct  6 01:08:25 2013
@@ -43,10 +43,9 @@ import org.apache.hive.service.cli.Table
 import org.apache.hive.service.cli.session.HiveSession;
 
 /**
- * HiveCommandOperation.
- *
+ * Executes a HiveCommand
  */
-public abstract class HiveCommandOperation extends ExecuteStatementOperation {
+public class HiveCommandOperation extends ExecuteStatementOperation {
   private CommandProcessorResponse response;
   private CommandProcessor commandProcessor;
   private TableSchema resultSchema = null;
@@ -58,8 +57,10 @@ public abstract class HiveCommandOperati
   private BufferedReader resultReader;
 
 
-  protected HiveCommandOperation(HiveSession parentSession, String statement, Map<String, String> confOverlay) {
+  protected HiveCommandOperation(HiveSession parentSession, String statement,
+      CommandProcessor commandProcessor, Map<String, String> confOverlay) {
     super(parentSession, statement, confOverlay);
+    this.commandProcessor = commandProcessor;
     setupSessionIO(parentSession.getSessionState());
   }
 
@@ -104,7 +105,7 @@ public abstract class HiveCommandOperati
       String[] tokens = statement.split("\\s");
       String commandArgs = command.substring(tokens[0].length()).trim();
 
-      response = getCommandProcessor().run(commandArgs);
+      response = commandProcessor.run(commandArgs);
       int returnCode = response.getResponseCode();
       String sqlState = response.getSQLState();
       String errorMessage = response.getErrorMessage();
@@ -201,12 +202,4 @@ public abstract class HiveCommandOperati
       resultReader = null;
     }
   }
-
-  protected CommandProcessor getCommandProcessor() {
-    return commandProcessor;
-  }
-
-  protected void setCommandProcessor(CommandProcessor commandProcessor) {
-    this.commandProcessor = commandProcessor;
-  }
 }

Modified: hive/trunk/service/src/java/org/apache/hive/service/cli/operation/OperationManager.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/operation/OperationManager.java?rev=1529558&r1=1529557&r2=1529558&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/operation/OperationManager.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/operation/OperationManager.java Sun Oct  6 01:08:25 2013
@@ -66,7 +66,8 @@ public class OperationManager extends Ab
   }
 
   public ExecuteStatementOperation newExecuteStatementOperation(HiveSession parentSession,
-      String statement, Map<String, String> confOverlay, boolean runAsync) {
+      String statement, Map<String, String> confOverlay, boolean runAsync)
+    throws HiveSQLException {
     ExecuteStatementOperation executeStatementOperation = ExecuteStatementOperation
         .newExecuteStatementOperation(parentSession, statement, confOverlay, runAsync);
     addOperation(executeStatementOperation);

Modified: hive/trunk/service/src/java/org/apache/hive/service/cli/operation/SetOperation.java
URL: http://svn.apache.org/viewvc/hive/trunk/service/src/java/org/apache/hive/service/cli/operation/SetOperation.java?rev=1529558&r1=1529557&r2=1529558&view=diff
==============================================================================
--- hive/trunk/service/src/java/org/apache/hive/service/cli/operation/SetOperation.java (original)
+++ hive/trunk/service/src/java/org/apache/hive/service/cli/operation/SetOperation.java Sun Oct  6 01:08:25 2013
@@ -1,38 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hive.service.cli.operation;
-
-import java.util.Map;
-
-import org.apache.hadoop.hive.ql.processors.SetProcessor;
-import org.apache.hive.service.cli.session.HiveSession;
-
-/**
- * HiveSetCommandOperation.
- *
- */
-public class SetOperation extends HiveCommandOperation {
-
-  protected SetOperation(HiveSession parentSession, String statement,
-      Map<String, String> confOverlay) {
-    super(parentSession, statement, confOverlay);
-    setCommandProcessor(new SetProcessor());
-  }
-
-}