You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by kg...@apache.org on 2018/02/07 08:52:09 UTC

[2/3] hive git commit: HIVE-17991: Remove CommandNeedRetryException (Zoltan Haindrich reviewed by Ashutosh Chauhan)

http://git-wip-us.apache.org/repos/asf/hive/blob/f7dea106/hcatalog/webhcat/java-client/src/test/java/org/apache/hive/hcatalog/api/repl/commands/TestCommands.java
----------------------------------------------------------------------
diff --git a/hcatalog/webhcat/java-client/src/test/java/org/apache/hive/hcatalog/api/repl/commands/TestCommands.java b/hcatalog/webhcat/java-client/src/test/java/org/apache/hive/hcatalog/api/repl/commands/TestCommands.java
index 6ae3d92..0417750 100644
--- a/hcatalog/webhcat/java-client/src/test/java/org/apache/hive/hcatalog/api/repl/commands/TestCommands.java
+++ b/hcatalog/webhcat/java-client/src/test/java/org/apache/hive/hcatalog/api/repl/commands/TestCommands.java
@@ -25,7 +25,6 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.cli.CliSessionState;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.ql.CommandNeedRetryException;
 import org.apache.hadoop.hive.ql.DriverFactory;
 import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
@@ -98,7 +97,7 @@ public class TestCommands {
   }
 
   @Test
-  public void testDropDatabaseCommand() throws HCatException, CommandNeedRetryException {
+  public void testDropDatabaseCommand() throws HCatException {
     String dbName = "cmd_testdb";
     int evid = 999;
     Command testCmd = new DropDatabaseCommand(dbName, evid);
@@ -130,7 +129,7 @@ public class TestCommands {
   }
 
   @Test
-  public void testDropTableCommand() throws HCatException, CommandNeedRetryException {
+  public void testDropTableCommand() throws HCatException {
     String dbName = "cmd_testdb";
     String tableName = "cmd_testtable";
     int evid = 789;
@@ -210,7 +209,7 @@ public class TestCommands {
   }
 
   @Test
-  public void testDropPartitionCommand() throws HCatException, CommandNeedRetryException, MetaException {
+  public void testDropPartitionCommand() throws HCatException, MetaException {
     String dbName = "cmd_testdb";
     String tableName = "cmd_testtable";
     int evid = 789;
@@ -302,7 +301,7 @@ public class TestCommands {
   }
 
   @Test
-  public void testDropTableCommand2() throws HCatException, CommandNeedRetryException, MetaException {
+  public void testDropTableCommand2() throws HCatException, MetaException {
     // Secondary DropTableCommand test for testing repl-drop-tables' effect on partitions inside a partitioned table
     // when there exist partitions inside the table which are older than the drop event.
     // Our goal is this : Create a table t, with repl.last.id=157, say.
@@ -373,7 +372,7 @@ public class TestCommands {
 
 
   @Test
-  public void testBasicReplEximCommands() throws IOException, CommandNeedRetryException {
+  public void testBasicReplEximCommands() throws IOException {
     // repl export, has repl.last.id and repl.scope=all in it
     // import repl dump, table has repl.last.id on it (will likely be 0)
     int evid = 111;
@@ -454,7 +453,7 @@ public class TestCommands {
   }
 
   @Test
-  public void testMetadataReplEximCommands() throws IOException, CommandNeedRetryException {
+  public void testMetadataReplEximCommands() throws IOException {
     // repl metadata export, has repl.last.id and repl.scope=metadata
     // import repl metadata dump, table metadata changed, allows override, has repl.last.id
     int evid = 222;
@@ -534,7 +533,7 @@ public class TestCommands {
 
 
   @Test
-  public void testNoopReplEximCommands() throws CommandNeedRetryException, IOException {
+  public void testNoopReplEximCommands() throws Exception {
     // repl noop export on non-existant table, has repl.noop, does not error
     // import repl noop dump, no error
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f7dea106/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestDDLWithRemoteMetastoreSecondNamenode.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestDDLWithRemoteMetastoreSecondNamenode.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestDDLWithRemoteMetastoreSecondNamenode.java
index cde728e..63a7313 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestDDLWithRemoteMetastoreSecondNamenode.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestDDLWithRemoteMetastoreSecondNamenode.java
@@ -18,7 +18,6 @@ package org.apache.hadoop.hive.ql;
 
 import java.io.IOException;
 import java.net.URI;
-import java.net.URISyntaxException;
 import java.util.HashMap;
 import junit.framework.JUnit4TestAdapter;
 import junit.framework.TestCase;
@@ -31,7 +30,6 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.Index;
-import org.apache.hadoop.hive.metastore.security.HadoopThriftAuthBridge;
 import org.apache.hadoop.hive.ql.exec.mr.ExecDriver;
 import org.apache.hadoop.hive.ql.metadata.*;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
@@ -143,7 +141,7 @@ public class TestDDLWithRemoteMetastoreSecondNamenode extends TestCase {
       }
   }
 
-  private void cleanup() throws CommandNeedRetryException {
+  private void cleanup() throws Exception {
       String[] srcidx = {Index1Name, Index2Name};
       for (String src : srcidx) {
         driver.run("DROP INDEX IF EXISTS " + src + " ON " + Table1Name);
@@ -159,7 +157,7 @@ public class TestDDLWithRemoteMetastoreSecondNamenode extends TestCase {
       }
   }
 
-  private void executeQuery(String query) throws CommandNeedRetryException {
+  private void executeQuery(String query) throws Exception {
     CommandProcessorResponse result =  driver.run(query);
     assertNotNull("driver.run() was expected to return result for query: " + query, result);
     assertEquals("Execution of (" + query + ") failed with exit status: "
@@ -173,7 +171,7 @@ public class TestDDLWithRemoteMetastoreSecondNamenode extends TestCase {
   }
 
   private void addPartitionAndCheck(Table table, String column,
-          String value, String location) throws CommandNeedRetryException, HiveException {
+      String value, String location) throws Exception {
     executeQuery("ALTER TABLE " + table.getTableName() +
             " ADD PARTITION (" + column + "='" + value + "')" +
             buildLocationClause(location));
@@ -197,8 +195,7 @@ public class TestDDLWithRemoteMetastoreSecondNamenode extends TestCase {
     }
   }
 
-  private void alterPartitionAndCheck(Table table, String column,
-      String value, String location) throws CommandNeedRetryException, HiveException {
+  private void alterPartitionAndCheck(Table table, String column, String value, String location) throws Exception {
     assertNotNull(location);
     executeQuery("ALTER TABLE " + table.getTableName() +
         " PARTITION (" + column + "='" + value + "')" +
@@ -219,12 +216,11 @@ public class TestDDLWithRemoteMetastoreSecondNamenode extends TestCase {
   }
 
   private Table createTableAndCheck(String tableName, String tableLocation)
-          throws CommandNeedRetryException, HiveException, URISyntaxException {
+      throws Exception {
     return createTableAndCheck(null, tableName, tableLocation);
   }
 
-  private Table createTableAndCheck(Table baseTable, String tableName, String tableLocation)
-          throws CommandNeedRetryException, HiveException, URISyntaxException {
+  private Table createTableAndCheck(Table baseTable, String tableName, String tableLocation) throws Exception {
     executeQuery("CREATE TABLE " + tableName + (baseTable == null ?
             " (col1 string, col2 string) PARTITIONED BY (p string) " :
             " LIKE " + baseTable.getTableName())
@@ -244,8 +240,7 @@ public class TestDDLWithRemoteMetastoreSecondNamenode extends TestCase {
     return table;
   }
 
-  private void createIndexAndCheck(Table table, String indexName, String indexLocation)
-          throws CommandNeedRetryException, HiveException, URISyntaxException {
+  private void createIndexAndCheck(Table table, String indexName, String indexLocation) throws Exception {
     executeQuery("CREATE INDEX " + indexName + " ON TABLE " + table.getTableName()
             + " (col1) AS 'COMPACT' WITH DEFERRED REBUILD "
             + buildLocationClause(indexLocation));
@@ -263,8 +258,7 @@ public class TestDDLWithRemoteMetastoreSecondNamenode extends TestCase {
     }
   }
 
-  private void createDatabaseAndCheck(String databaseName, String databaseLocation)
-          throws CommandNeedRetryException, HiveException, URISyntaxException {
+  private void createDatabaseAndCheck(String databaseName, String databaseLocation) throws Exception {
     executeQuery("CREATE DATABASE " + databaseName + buildLocationClause(databaseLocation));
     Database database = db.getDatabase(databaseName);
     assertNotNull("Database object is expected for " + databaseName , database);

http://git-wip-us.apache.org/repos/asf/hive/blob/f7dea106/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
index d1d2d1f..d763666 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
@@ -50,8 +50,6 @@ import org.apache.hadoop.hive.metastore.messaging.event.filters.AndFilter;
 import org.apache.hadoop.hive.metastore.messaging.event.filters.DatabaseAndTableFilter;
 import org.apache.hadoop.hive.metastore.messaging.event.filters.EventBoundaryFilter;
 import org.apache.hadoop.hive.metastore.messaging.event.filters.MessageFormatFilter;
-import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
-import org.apache.hadoop.hive.ql.CommandNeedRetryException;
 import org.apache.hadoop.hive.ql.DriverFactory;
 import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.exec.repl.ReplDumpWork;
@@ -80,10 +78,8 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
-import java.util.Map;
-
-import static junit.framework.Assert.assertFalse;
-import static junit.framework.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
@@ -587,22 +583,19 @@ public class TestReplicationScenarios {
           // getTable is invoked after fetching the table names
           injectionPathCalled = true;
           Thread t = new Thread(new Runnable() {
+            @Override
             public void run() {
-              try {
-                LOG.info("Entered new thread");
-                IDriver driver2 = DriverFactory.newDriver(hconf);
-                SessionState.start(new CliSessionState(hconf));
-                CommandProcessorResponse ret = driver2.run("ALTER TABLE " + dbName + ".ptned PARTITION (b=1) RENAME TO PARTITION (b=10)");
-                success = (ret.getException() == null);
-                assertFalse(success);
-                ret = driver2.run("ALTER TABLE " + dbName + ".ptned RENAME TO " + dbName + ".ptned_renamed");
-                success = (ret.getException() == null);
-                assertFalse(success);
-                LOG.info("Exit new thread success - {}", success);
-              } catch (CommandNeedRetryException e) {
-                LOG.info("Hit Exception {} from new thread", e.getMessage());
-                throw new RuntimeException(e);
-              }
+              LOG.info("Entered new thread");
+              IDriver driver2 = DriverFactory.newDriver(hconf);
+              SessionState.start(new CliSessionState(hconf));
+              CommandProcessorResponse ret =
+                  driver2.run("ALTER TABLE " + dbName + ".ptned PARTITION (b=1) RENAME TO PARTITION (b=10)");
+              success = (ret.getException() == null);
+              assertFalse(success);
+              ret = driver2.run("ALTER TABLE " + dbName + ".ptned RENAME TO " + dbName + ".ptned_renamed");
+              success = (ret.getException() == null);
+              assertFalse(success);
+              LOG.info("Exit new thread success - {}", success);
             }
           });
           t.start();
@@ -662,19 +655,15 @@ public class TestReplicationScenarios {
           // getTable is invoked after fetching the table names
           injectionPathCalled = true;
           Thread t = new Thread(new Runnable() {
+            @Override
             public void run() {
-              try {
-                LOG.info("Entered new thread");
-                IDriver driver2 = DriverFactory.newDriver(hconf);
-                SessionState.start(new CliSessionState(hconf));
-                CommandProcessorResponse ret = driver2.run("DROP TABLE " + dbName + ".ptned");
-                success = (ret.getException() == null);
-                assertTrue(success);
-                LOG.info("Exit new thread success - {}", success);
-              } catch (CommandNeedRetryException e) {
-                LOG.info("Hit Exception {} from new thread", e.getMessage());
-                throw new RuntimeException(e);
-              }
+              LOG.info("Entered new thread");
+              IDriver driver2 = DriverFactory.newDriver(hconf);
+              SessionState.start(new CliSessionState(hconf));
+              CommandProcessorResponse ret = driver2.run("DROP TABLE " + dbName + ".ptned");
+              success = (ret.getException() == null);
+              assertTrue(success);
+              LOG.info("Exit new thread success - {}", success);
             }
           });
           t.start();
@@ -3124,7 +3113,7 @@ public class TestReplicationScenarios {
       List<SQLNotNullConstraint> nns = metaStoreClientMirror.getNotNullConstraints(new NotNullConstraintsRequest(dbName+ "_dupe" , "tbl6"));
       assertEquals(nns.size(), 1);
       nnName = nns.get(0).getNn_name();
-      
+
     } catch (TException te) {
       assertNull(te);
     }
@@ -3616,12 +3605,7 @@ public class TestReplicationScenarios {
   private String getResult(int rowNum, int colNum, boolean reuse, IDriver myDriver) throws IOException {
     if (!reuse) {
       lastResults = new ArrayList<String>();
-      try {
-        myDriver.getResults(lastResults);
-      } catch (CommandNeedRetryException e) {
-        e.printStackTrace();
-        throw new RuntimeException(e);
-      }
+      myDriver.getResults(lastResults);
     }
     // Split around the 'tab' character
     return (lastResults.get(rowNum).split("\\t"))[colNum];
@@ -3646,12 +3630,7 @@ public class TestReplicationScenarios {
 
   private List<String> getOutput(IDriver myDriver) throws IOException {
     List<String> results = new ArrayList<>();
-    try {
-      myDriver.getResults(results);
-    } catch (CommandNeedRetryException e) {
-      LOG.warn(e.getMessage(),e);
-      throw new RuntimeException(e);
-    }
+    myDriver.getResults(results);
     return results;
   }
 
@@ -3772,19 +3751,10 @@ public class TestReplicationScenarios {
 
   private static boolean run(String cmd, boolean errorOnFail, IDriver myDriver) throws RuntimeException {
     boolean success = false;
-    try {
-      CommandProcessorResponse ret = myDriver.run(cmd);
-      success = ((ret.getException() == null) && (ret.getErrorMessage() == null));
-      if (!success){
-        LOG.warn("Error {} : {} running [{}].", ret.getErrorCode(), ret.getErrorMessage(), cmd);
-      }
-    } catch (CommandNeedRetryException e) {
-      if (errorOnFail){
-        throw new RuntimeException(e);
-      } else {
-        LOG.warn(e.getMessage(),e);
-        // do nothing else
-      }
+    CommandProcessorResponse ret = myDriver.run(cmd);
+    success = ((ret.getException() == null) && (ret.getErrorMessage() == null));
+    if (!success) {
+      LOG.warn("Error {} : {} running [{}].", ret.getErrorCode(), ret.getErrorMessage(), cmd);
     }
     return success;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/f7dea106/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java
index dd6fa42..33e5157 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/WarehouseInstance.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
 import org.apache.hadoop.hive.metastore.MetaStoreTestUtils;
-import org.apache.hadoop.hive.ql.CommandNeedRetryException;
 import org.apache.hadoop.hive.ql.DriverFactory;
 import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.exec.repl.ReplDumpWork;
@@ -160,12 +159,7 @@ class WarehouseInstance implements Closeable {
   private String row0Result(int colNum, boolean reuse) throws IOException {
     if (!reuse) {
       lastResults = new ArrayList<>();
-      try {
-        driver.getResults(lastResults);
-      } catch (CommandNeedRetryException e) {
-        e.printStackTrace();
-        throw new RuntimeException(e);
-      }
+      driver.getResults(lastResults);
     }
     // Split around the 'tab' character
     return (lastResults.get(0).split("\\t"))[colNum];
@@ -265,12 +259,7 @@ class WarehouseInstance implements Closeable {
 
   List<String> getOutput() throws IOException {
     List<String> results = new ArrayList<>();
-    try {
-      driver.getResults(results);
-    } catch (CommandNeedRetryException e) {
-      logger.warn(e.getMessage(), e);
-      throw new RuntimeException(e);
-    }
+    driver.getResults(results);
     return results;
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f7dea106/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java
index 6f2405c..bc2a34a 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerCheckInvocation.java
@@ -37,7 +37,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hadoop.hive.ql.CommandNeedRetryException;
 import org.apache.hadoop.hive.ql.DriverFactory;
 import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
@@ -114,7 +113,7 @@ public class TestHiveAuthorizerCheckInvocation {
         "stored as orc TBLPROPERTIES ('transactional'='true')");
   }
 
-  private static void runCmd(String cmd) throws CommandNeedRetryException {
+  private static void runCmd(String cmd) throws Exception {
     CommandProcessorResponse resp = driver.run(cmd);
     assertEquals(0, resp.getResponseCode());
   }
@@ -131,8 +130,7 @@ public class TestHiveAuthorizerCheckInvocation {
   }
 
   @Test
-  public void testInputSomeColumnsUsed() throws HiveAuthzPluginException, HiveAccessControlException,
-      CommandNeedRetryException {
+  public void testInputSomeColumnsUsed() throws Exception {
 
     reset(mockedAuthorizer);
     int status = driver.compile("select i from " + tableName
@@ -148,8 +146,7 @@ public class TestHiveAuthorizerCheckInvocation {
   }
 
   @Test
-  public void testInputSomeColumnsUsedView() throws HiveAuthzPluginException, HiveAccessControlException,
-  CommandNeedRetryException {
+  public void testInputSomeColumnsUsedView() throws Exception {
 
     reset(mockedAuthorizer);
     int status = driver.compile("select i from " + viewName
@@ -165,15 +162,14 @@ public class TestHiveAuthorizerCheckInvocation {
   }
 
   @Test
-  public void testInputSomeColumnsUsedJoin() throws HiveAuthzPluginException, HiveAccessControlException,
-  CommandNeedRetryException {
-    
+  public void testInputSomeColumnsUsedJoin() throws Exception {
+
     reset(mockedAuthorizer);
     int status = driver.compile("select " + viewName + ".i, " + tableName + ".city from "
         + viewName + " join " + tableName + " on " + viewName + ".city = " + tableName
         + ".city where " + tableName + ".k = 'X'");
     assertEquals(0, status);
-    
+
     List<HivePrivilegeObject> inputs = getHivePrivilegeObjectInputs().getLeft();
     Collections.sort(inputs);
     assertEquals(inputs.size(), 2);
@@ -194,8 +190,7 @@ public class TestHiveAuthorizerCheckInvocation {
   }
 
   @Test
-  public void testInputAllColumnsUsed() throws HiveAuthzPluginException, HiveAccessControlException,
-      CommandNeedRetryException {
+  public void testInputAllColumnsUsed() throws Exception {
 
     reset(mockedAuthorizer);
     int status = driver.compile("select * from " + tableName + " order by i");
@@ -210,15 +205,13 @@ public class TestHiveAuthorizerCheckInvocation {
   }
 
   @Test
-  public void testCreateTableWithDb() throws HiveAuthzPluginException, HiveAccessControlException,
-      CommandNeedRetryException {
+  public void testCreateTableWithDb() throws Exception {
     final String newTable = "ctTableWithDb";
     checkCreateViewOrTableWithDb(newTable, "create table " + dbName + "." + newTable + "(i int)");
   }
 
   @Test
-  public void testCreateViewWithDb() throws HiveAuthzPluginException, HiveAccessControlException,
-      CommandNeedRetryException {
+  public void testCreateViewWithDb() throws Exception {
     final String newTable = "ctViewWithDb";
     checkCreateViewOrTableWithDb(newTable, "create table " + dbName + "." + newTable + "(i int)");
   }
@@ -251,8 +244,7 @@ public class TestHiveAuthorizerCheckInvocation {
   }
 
   @Test
-  public void testInputNoColumnsUsed() throws HiveAuthzPluginException, HiveAccessControlException,
-      CommandNeedRetryException {
+  public void testInputNoColumnsUsed() throws Exception {
 
     reset(mockedAuthorizer);
     int status = driver.compile("describe " + tableName);
@@ -265,8 +257,7 @@ public class TestHiveAuthorizerCheckInvocation {
   }
 
   @Test
-  public void testPermFunction() throws HiveAuthzPluginException, HiveAccessControlException,
-      CommandNeedRetryException {
+  public void testPermFunction() throws Exception {
 
     reset(mockedAuthorizer);
     final String funcName = "testauthfunc1";
@@ -296,8 +287,7 @@ public class TestHiveAuthorizerCheckInvocation {
   }
 
   @Test
-  public void testTempFunction() throws HiveAuthzPluginException, HiveAccessControlException,
-      CommandNeedRetryException {
+  public void testTempFunction() throws Exception {
 
     reset(mockedAuthorizer);
     final String funcName = "testAuthFunc2";
@@ -313,8 +303,7 @@ public class TestHiveAuthorizerCheckInvocation {
   }
 
   @Test
-  public void testUpdateSomeColumnsUsed() throws HiveAuthzPluginException,
-      HiveAccessControlException, CommandNeedRetryException {
+  public void testUpdateSomeColumnsUsed() throws Exception {
     reset(mockedAuthorizer);
     int status = driver.compile("update " + acidTableName + " set i = 5 where j = 3");
     assertEquals(0, status);
@@ -333,8 +322,7 @@ public class TestHiveAuthorizerCheckInvocation {
   }
 
   @Test
-  public void testUpdateSomeColumnsUsedExprInSet() throws HiveAuthzPluginException,
-      HiveAccessControlException, CommandNeedRetryException {
+  public void testUpdateSomeColumnsUsedExprInSet() throws Exception {
     reset(mockedAuthorizer);
     int status = driver.compile("update " + acidTableName + " set i = 5, j = k where j = 3");
     assertEquals(0, status);
@@ -355,8 +343,7 @@ public class TestHiveAuthorizerCheckInvocation {
   }
 
   @Test
-  public void testDelete() throws HiveAuthzPluginException,
-      HiveAccessControlException, CommandNeedRetryException {
+  public void testDelete() throws Exception {
     reset(mockedAuthorizer);
     int status = driver.compile("delete from " + acidTableName + " where j = 3");
     assertEquals(0, status);
@@ -370,8 +357,7 @@ public class TestHiveAuthorizerCheckInvocation {
   }
 
   @Test
-  public void testShowTables() throws HiveAuthzPluginException,
-      HiveAccessControlException, CommandNeedRetryException {
+  public void testShowTables() throws Exception {
     reset(mockedAuthorizer);
     int status = driver.compile("show tables");
     assertEquals(0, status);
@@ -384,8 +370,7 @@ public class TestHiveAuthorizerCheckInvocation {
   }
 
   @Test
-  public void testDescDatabase() throws HiveAuthzPluginException,
-      HiveAccessControlException, CommandNeedRetryException {
+  public void testDescDatabase() throws Exception {
     reset(mockedAuthorizer);
     int status = driver.compile("describe database " + dbName);
     assertEquals(0, status);
@@ -411,8 +396,7 @@ public class TestHiveAuthorizerCheckInvocation {
   }
 
   @Test
-  public void testReplDump() throws HiveAuthzPluginException, HiveAccessControlException,
-      CommandNeedRetryException {
+  public void testReplDump() throws Exception {
 
     resetAuthorizer();
     int status = driver.compile("repl dump " + dbName);

http://git-wip-us.apache.org/repos/asf/hive/blob/f7dea106/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerShowFilters.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerShowFilters.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerShowFilters.java
index 2be86f8..8981223 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerShowFilters.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerShowFilters.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hive.ql.security.authorization.plugin;
 import static org.junit.Assert.assertEquals;
 import static org.mockito.Matchers.any;
 
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -30,7 +29,6 @@ import java.util.List;
 import org.apache.hadoop.hive.UtilsForTest;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hadoop.hive.ql.CommandNeedRetryException;
 import org.apache.hadoop.hive.ql.DriverFactory;
 import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
@@ -90,7 +88,7 @@ public class TestHiveAuthorizerShowFilters {
         return filteredResults;
       }
     }
-    
+
     @Override
     public HiveAuthorizer createHiveAuthorizer(HiveMetastoreClientFactory metastoreClientFactory,
         HiveConf conf, HiveAuthenticationProvider authenticator, HiveAuthzSessionContext ctx) {
@@ -153,46 +151,40 @@ public class TestHiveAuthorizerShowFilters {
   }
 
   @Test
-  public void testShowDatabasesAll() throws HiveAuthzPluginException, HiveAccessControlException,
-      CommandNeedRetryException, IOException {
+  public void testShowDatabasesAll() throws Exception {
     runShowDbTest(AllDbs);
   }
 
   @Test
-  public void testShowDatabasesSelected() throws HiveAuthzPluginException,
-      HiveAccessControlException, CommandNeedRetryException, IOException {
+  public void testShowDatabasesSelected() throws Exception {
     setFilteredResults(HivePrivilegeObjectType.DATABASE, dbName2);
     runShowDbTest(Arrays.asList(dbName2));
   }
 
-  private void runShowDbTest(List<String> expectedDbList) throws HiveAuthzPluginException,
-      HiveAccessControlException, CommandNeedRetryException, IOException {
+  private void runShowDbTest(List<String> expectedDbList) throws Exception {
     runCmd("show databases");
     verifyAllDb();
     assertEquals("filtered result check ", expectedDbList, getSortedResults());
   }
 
   @Test
-  public void testShowTablesAll() throws HiveAuthzPluginException, HiveAccessControlException,
-      CommandNeedRetryException, IOException {
+  public void testShowTablesAll() throws Exception {
     runShowTablesTest(AllTables);
   }
 
   @Test
-  public void testShowTablesSelected() throws HiveAuthzPluginException, HiveAccessControlException,
-      CommandNeedRetryException, IOException {
+  public void testShowTablesSelected() throws Exception {
     setFilteredResults(HivePrivilegeObjectType.TABLE_OR_VIEW, tableName2);
     runShowTablesTest(Arrays.asList(tableName2));
   }
 
-  private void runShowTablesTest(List<String> expectedTabs) throws IOException,
-      CommandNeedRetryException, HiveAuthzPluginException, HiveAccessControlException {
+  private void runShowTablesTest(List<String> expectedTabs) throws Exception {
     runCmd("show tables");
     verifyAllTables();
     assertEquals("filtered result check ", expectedTabs, getSortedResults());
   }
 
-  private List<String> getSortedResults() throws IOException, CommandNeedRetryException {
+  private List<String> getSortedResults() throws Exception {
     List<String> res = new ArrayList<String>();
     // set results to be returned
     driver.getResults(res);
@@ -262,7 +254,7 @@ public class TestHiveAuthorizerShowFilters {
     }
   }
 
-  private static void runCmd(String cmd) throws CommandNeedRetryException {
+  private static void runCmd(String cmd) throws Exception {
     CommandProcessorResponse resp = driver.run(cmd);
     assertEquals(0, resp.getResponseCode());
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/f7dea106/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java
index 1305902..a5e6293 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java
@@ -55,7 +55,6 @@ import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
 import org.apache.hadoop.hive.metastore.txn.TxnDbUtil;
 import org.apache.hadoop.hive.metastore.txn.TxnStore;
 import org.apache.hadoop.hive.metastore.txn.TxnUtils;
-import org.apache.hadoop.hive.ql.CommandNeedRetryException;
 import org.apache.hadoop.hive.ql.DriverFactory;
 import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.io.AcidInputFormat;
@@ -1425,7 +1424,7 @@ public class TestCompactor {
    * Execute Hive CLI statement
    * @param cmd arbitrary statement to execute
    */
-  static void executeStatementOnDriver(String cmd, IDriver driver) throws IOException, CommandNeedRetryException {
+  static void executeStatementOnDriver(String cmd, IDriver driver) throws Exception {
     LOG.debug("Executing: " + cmd);
     CommandProcessorResponse cpr = driver.run(cmd);
     if(cpr.getResponseCode() != 0) {

http://git-wip-us.apache.org/repos/asf/hive/blob/f7dea106/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
----------------------------------------------------------------------
diff --git a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
index 2bc33bd..2d0aca0 100644
--- a/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
+++ b/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
@@ -1313,13 +1313,7 @@ public class QTestUtil {
   }
 
   public int execute(String tname) {
-    try {
-      return drv.run(qMap.get(tname)).getResponseCode();
-    } catch (CommandNeedRetryException e) {
-      LOG.error("driver failed to run the command: " + tname + " due to the exception: ", e);
-      e.printStackTrace();
-      return -1;
-    }
+    return drv.run(qMap.get(tname)).getResponseCode();
   }
 
   public int executeClient(String tname1, String tname2) {

http://git-wip-us.apache.org/repos/asf/hive/blob/f7dea106/ql/src/java/org/apache/hadoop/hive/ql/CommandNeedRetryException.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/CommandNeedRetryException.java b/ql/src/java/org/apache/hadoop/hive/ql/CommandNeedRetryException.java
deleted file mode 100644
index 1b60005..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/CommandNeedRetryException.java
+++ /dev/null
@@ -1,40 +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.hadoop.hive.ql;
-
-public class CommandNeedRetryException extends Exception {
-
-  private static final long serialVersionUID = 1L;
-
-  public CommandNeedRetryException() {
-    super();
-  }
-
-  public CommandNeedRetryException(String message) {
-    super(message);
-  }
-
-  public CommandNeedRetryException(Throwable cause) {
-    super(cause);
-  }
-
-  public CommandNeedRetryException(String message, Throwable cause) {
-    super(message, cause);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hive/blob/f7dea106/ql/src/java/org/apache/hadoop/hive/ql/Context.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Context.java b/ql/src/java/org/apache/hadoop/hive/ql/Context.java
index 820fbf0..dba2dbb 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Context.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Context.java
@@ -100,8 +100,6 @@ public class Context {
   protected String cboInfo;
   protected boolean cboSucceeded;
   protected String cmd = "";
-  // number of previous attempts
-  protected int tryCount = 0;
   private TokenRewriteStream tokenRewriteStream;
   // Holds the qualified name to tokenRewriteStream for the views
   // referenced by the query. This is used to rewrite the view AST
@@ -178,6 +176,7 @@ public class Context {
     DestClausePrefix(String prefix) {
       this.prefix = prefix;
     }
+    @Override
     public String toString() {
       return prefix;
     }
@@ -941,14 +940,6 @@ public class Context {
     this.needLockMgr = needLockMgr;
   }
 
-  public int getTryCount() {
-    return tryCount;
-  }
-
-  public void setTryCount(int tryCount) {
-    this.tryCount = tryCount;
-  }
-
   public String getCboInfo() {
     return cboInfo;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/f7dea106/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
index 74595b0..2d7e459 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
@@ -158,7 +158,6 @@ public class Driver implements IDriver {
 
   // A limit on the number of threads that can be launched
   private int maxthreads;
-  private int tryCount = Integer.MAX_VALUE;
 
   private String userName;
 
@@ -577,7 +576,6 @@ public class Driver implements IDriver {
         setTriggerContext(queryId);
       }
 
-      ctx.setTryCount(getTryCount());
       ctx.setCmd(command);
       ctx.setHDFSCleanup(true);
 
@@ -1377,19 +1375,16 @@ public class Driver implements IDriver {
 
   @Override
 
-  public CommandProcessorResponse run(String command)
-      throws CommandNeedRetryException {
+  public CommandProcessorResponse run(String command) {
     return run(command, false);
   }
 
   @Override
-  public CommandProcessorResponse run()
-      throws CommandNeedRetryException {
+  public CommandProcessorResponse run() {
     return run(null, true);
   }
 
-  public CommandProcessorResponse run(String command, boolean alreadyCompiled)
-        throws CommandNeedRetryException {
+  public CommandProcessorResponse run(String command, boolean alreadyCompiled) {
 
     try {
       runInternal(command, alreadyCompiled);
@@ -1579,8 +1574,7 @@ public class Driver implements IDriver {
     return compileLock;
   }
 
-  private void runInternal(String command, boolean alreadyCompiled)
-      throws CommandNeedRetryException, CommandProcessorResponse {
+  private void runInternal(String command, boolean alreadyCompiled) throws CommandProcessorResponse {
     errorMessage = null;
     SQLState = null;
     downstreamError = null;
@@ -1794,7 +1788,7 @@ public class Driver implements IDriver {
     return new CommandProcessorResponse(ret, errorMessage, SQLState, downstreamError);
   }
 
-  private void execute() throws CommandNeedRetryException, CommandProcessorResponse {
+  private void execute() throws CommandProcessorResponse {
     PerfLogger perfLogger = SessionState.getPerfLogger();
     perfLogger.PerfLogBegin(CLASS_NAME, PerfLogger.DRIVER_EXECUTE);
 
@@ -1949,13 +1943,6 @@ public class Driver implements IDriver {
         checkInterrupted("when checking the execution result.", hookContext, perfLogger);
 
         if (exitVal != 0) {
-          if (tsk.ifRetryCmdWhenFail()) {
-            driverCxt.shutdown();
-            // in case we decided to run everything in local mode, restore the
-            // the jobtracker setting to its initial value
-            ctx.restoreOriginalTracker();
-            throw new CommandNeedRetryException();
-          }
           Task<? extends Serializable> backupTask = tsk.getAndInitBackupTask();
           if (backupTask != null) {
             setErrorMsgAndDetail(exitVal, result.getTaskError(), tsk);
@@ -2057,9 +2044,6 @@ public class Driver implements IDriver {
         SessionState.get().getHiveHistory().printRowCount(queryId);
       }
       releasePlan(plan);
-    } catch (CommandNeedRetryException e) {
-      executionError = true;
-      throw e;
     } catch (CommandProcessorResponse cpr) {
       executionError = true;
       throw cpr;
@@ -2276,7 +2260,7 @@ public class Driver implements IDriver {
 
   @SuppressWarnings("unchecked")
   @Override
-  public boolean getResults(List res) throws IOException, CommandNeedRetryException {
+  public boolean getResults(List res) throws IOException {
     if (lDrvState.driverState == DriverState.DESTROYED || lDrvState.driverState == DriverState.CLOSED) {
       throw new IOException("FAILED: query has been cancelled, closed, or destroyed.");
     }
@@ -2359,15 +2343,6 @@ public class Driver implements IDriver {
     }
   }
 
-  public int getTryCount() {
-    return tryCount;
-  }
-
-  @Override
-  public void setTryCount(int tryCount) {
-    this.tryCount = tryCount;
-  }
-
   // DriverContext could be released in the query and close processes at same
   // time, which needs to be thread protected.
   private void releaseDriverContext() {

http://git-wip-us.apache.org/repos/asf/hive/blob/f7dea106/ql/src/java/org/apache/hadoop/hive/ql/IDriver.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/IDriver.java b/ql/src/java/org/apache/hadoop/hive/ql/IDriver.java
index 6280be0..d4494cc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/IDriver.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/IDriver.java
@@ -41,15 +41,13 @@ public interface IDriver extends CommandProcessor {
 
   void setOperationId(String guid64);
 
-  void setTryCount(int maxValue);
-
-  CommandProcessorResponse run() throws CommandNeedRetryException;
+  CommandProcessorResponse run();
   @Override
-  CommandProcessorResponse run(String command) throws CommandNeedRetryException;
+  CommandProcessorResponse run(String command);
 
 
   // create some "cover" to the result?
-  boolean getResults(List res) throws IOException, CommandNeedRetryException;
+  boolean getResults(List res) throws IOException;
 
   void setMaxRows(int maxRows);
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f7dea106/ql/src/java/org/apache/hadoop/hive/ql/QueryDisplay.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/QueryDisplay.java b/ql/src/java/org/apache/hadoop/hive/ql/QueryDisplay.java
index cc67b87..9a77c29 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/QueryDisplay.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/QueryDisplay.java
@@ -79,7 +79,6 @@ public class QueryDisplay {
     private StageType taskType;
     private String name;
     private boolean requireLock;
-    private boolean retryIfFail;
     private String statusMessage;
 
     // required for jackson
@@ -92,7 +91,6 @@ public class QueryDisplay {
       taskType = task.getType();
       name = task.getName();
       requireLock = task.requireLock();
-      retryIfFail = task.ifRetryCmdWhenFail();
     }
     @JsonIgnore
     public synchronized String getStatus() {
@@ -147,10 +145,6 @@ public class QueryDisplay {
     public synchronized boolean isRequireLock() {
       return requireLock;
     }
-    @JsonIgnore
-    public synchronized boolean isRetryIfFail() {
-      return retryIfFail;
-    }
 
     public synchronized String getExternalHandle() {
       return externalHandle;

http://git-wip-us.apache.org/repos/asf/hive/blob/f7dea106/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchTask.java
index a7dace9..090a188 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FetchTask.java
@@ -24,7 +24,6 @@ import java.util.List;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hive.ql.CommandNeedRetryException;
 import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.DriverContext;
 import org.apache.hadoop.hive.ql.QueryPlan;
@@ -130,7 +129,7 @@ public class FetchTask extends Task<FetchWork> implements Serializable {
     this.maxRows = maxRows;
   }
 
-  public boolean fetch(List res) throws IOException, CommandNeedRetryException {
+  public boolean fetch(List res) throws IOException {
     sink.reset(res);
     int rowsRet = work.getLeastNumRows();
     if (rowsRet <= 0) {
@@ -145,7 +144,7 @@ public class FetchTask extends Task<FetchWork> implements Serializable {
       while (sink.getNumRows() < rowsRet) {
         if (!fetch.pushRow()) {
           if (work.getLeastNumRows() > 0) {
-            throw new CommandNeedRetryException();
+            throw new HiveException("leastNumRows check failed");
           }
 
           // Closing the operator can sometimes yield more rows (HIVE-11892)
@@ -156,8 +155,6 @@ public class FetchTask extends Task<FetchWork> implements Serializable {
         fetched = true;
       }
       return true;
-    } catch (CommandNeedRetryException e) {
-      throw e;
     } catch (IOException e) {
       throw e;
     } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/hive/blob/f7dea106/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java
index 7124c89..a8d851f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Task.java
@@ -579,14 +579,6 @@ public abstract class Task<T extends Serializable> implements Serializable, Node
     return false;
   }
 
-  public boolean ifRetryCmdWhenFail() {
-    return retryCmdWhenFail;
-  }
-
-  public void setRetryCmdWhenFail(boolean retryCmdWhenFail) {
-    this.retryCmdWhenFail = retryCmdWhenFail;
-  }
-
   public QueryPlan getQueryPlan() {
     return queryPlan;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/f7dea106/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GlobalLimitOptimizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GlobalLimitOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GlobalLimitOptimizer.java
index e656eb0..bd0cbab 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GlobalLimitOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/GlobalLimitOptimizer.java
@@ -75,7 +75,7 @@ public class GlobalLimitOptimizer extends Transform {
     // The query only qualifies when there are only one top operator
     // and there is no transformer or UDTF and no block sampling
     // is used.
-    if (ctx.getTryCount() == 0 && topOps.size() == 1
+    if (topOps.size() == 1
         && !globalLimitCtx.ifHasTransformOrUDTF() &&
         nameToSplitSample.isEmpty()) {
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f7dea106/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java
index f01b497..63b13c8 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ExplainSemanticAnalyzer.java
@@ -33,7 +33,6 @@ import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.ql.CommandNeedRetryException;
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.ql.QueryState;
@@ -147,8 +146,6 @@ public class ExplainSemanticAnalyzer extends BaseSemanticAnalyzer {
         config.setOpIdToRuntimeNumRows(aggregateStats(config.getExplainRootPath()));
       } catch (IOException e1) {
         throw new SemanticException(e1);
-      } catch (CommandNeedRetryException e) {
-        throw new SemanticException(e);
       }
       ctx.resetOpContext();
       ctx.resetStream();

http://git-wip-us.apache.org/repos/asf/hive/blob/f7dea106/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
index 0c1c4e0..92d29e3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TaskCompiler.java
@@ -47,8 +47,6 @@ import org.apache.hadoop.hive.ql.exec.TableScanOperator;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.exec.TaskFactory;
 import org.apache.hadoop.hive.ql.exec.Utilities;
-import org.apache.hadoop.hive.ql.exec.mr.ExecDriver;
-import org.apache.hadoop.hive.ql.exec.spark.SparkTask;
 import org.apache.hadoop.hive.ql.hooks.ReadEntity;
 import org.apache.hadoop.hive.ql.hooks.WriteEntity;
 import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat;
@@ -72,7 +70,6 @@ import org.apache.hadoop.hive.ql.plan.MoveWork;
 import org.apache.hadoop.hive.ql.plan.PlanUtils;
 import org.apache.hadoop.hive.ql.plan.StatsWork;
 import org.apache.hadoop.hive.ql.plan.TableDesc;
-import org.apache.hadoop.hive.ql.session.LineageState;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.ql.session.SessionState.LogHelper;
 import org.apache.hadoop.hive.serde.serdeConstants;
@@ -362,14 +359,6 @@ public abstract class TaskCompiler {
     if (globalLimitCtx.isEnable() && globalLimitCtx.getLastReduceLimitDesc() != null) {
       LOG.info("set least row check for LimitDesc: " + globalLimitCtx.getGlobalLimit());
       globalLimitCtx.getLastReduceLimitDesc().setLeastRows(globalLimitCtx.getGlobalLimit());
-      List<ExecDriver> mrTasks = Utilities.getMRTasks(rootTasks);
-      for (ExecDriver tsk : mrTasks) {
-        tsk.setRetryCmdWhenFail(true);
-      }
-      List<SparkTask> sparkTasks = Utilities.getSparkTasks(rootTasks);
-      for (SparkTask sparkTask : sparkTasks) {
-        sparkTask.setRetryCmdWhenFail(true);
-      }
     }
 
     Interner<TableDesc> interner = Interners.newStrongInterner();

http://git-wip-us.apache.org/repos/asf/hive/blob/f7dea106/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessor.java
index 3624d08..c753264 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/processors/CommandProcessor.java
@@ -18,8 +18,6 @@
 
 package org.apache.hadoop.hive.ql.processors;
 
-import org.apache.hadoop.hive.ql.CommandNeedRetryException;
-
 public interface CommandProcessor {
-  CommandProcessorResponse run(String command) throws CommandNeedRetryException;
+  CommandProcessorResponse run(String command);
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/f7dea106/ql/src/java/org/apache/hadoop/hive/ql/processors/CompileProcessor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/processors/CompileProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/processors/CompileProcessor.java
index 07d70ab..fad4f52 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/processors/CompileProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/processors/CompileProcessor.java
@@ -33,7 +33,6 @@ import org.apache.commons.compress.archivers.jar.JarArchiveOutputStream;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.hive.conf.HiveVariableSource;
 import org.apache.hadoop.hive.conf.VariableSubstitution;
-import org.apache.hadoop.hive.ql.CommandNeedRetryException;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
 import org.apache.hadoop.hive.ql.session.SessionState;
@@ -104,7 +103,7 @@ public class CompileProcessor implements CommandProcessor {
    * @return CommandProcessorResponse with 0 for success and 1 for failure
    */
   @Override
-  public CommandProcessorResponse run(String command) throws CommandNeedRetryException {
+  public CommandProcessorResponse run(String command) {
     SessionState ss = SessionState.get();
     this.command = command;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/f7dea106/ql/src/java/org/apache/hadoop/hive/ql/processors/CryptoProcessor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/processors/CryptoProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/processors/CryptoProcessor.java
index 6825dd8..d1202f9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/processors/CryptoProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/processors/CryptoProcessor.java
@@ -28,7 +28,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.ql.CommandNeedRetryException;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.shims.HadoopShims;
 
@@ -82,7 +81,7 @@ public class CryptoProcessor implements CommandProcessor {
   }
 
   @Override
-  public CommandProcessorResponse run(String command) throws CommandNeedRetryException {
+  public CommandProcessorResponse run(String command) {
     String[] args = command.split("\\s+");
 
     if (args.length < 1) {

http://git-wip-us.apache.org/repos/asf/hive/blob/f7dea106/ql/src/java/org/apache/hadoop/hive/ql/processors/DfsProcessor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/processors/DfsProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/processors/DfsProcessor.java
index 2f288ce..62a1725 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/processors/DfsProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/processors/DfsProcessor.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.hive.conf.HiveVariableSource;
 import org.apache.hadoop.hive.conf.VariableSubstitution;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Schema;
-import org.apache.hadoop.hive.ql.CommandNeedRetryException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
 import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.hive.ql.session.SessionState.LogHelper;
@@ -102,7 +102,7 @@ public class DfsProcessor implements CommandProcessor {
     }
   }
 
-  private String[] splitCmd(String command) throws CommandNeedRetryException {
+  private String[] splitCmd(String command) throws HiveException {
 
     ArrayList<String> paras = new ArrayList<String>();
     int cmdLng = command.length();
@@ -151,8 +151,9 @@ public class DfsProcessor implements CommandProcessor {
     }
 
     if ((int) y != 0) {
-      console.printError("Syntax error on hadoop options: dfs " + command);
-      throw new CommandNeedRetryException();
+      String message = "Syntax error on hadoop options: dfs " + command;
+      console.printError(message);
+      throw new HiveException(message);
     }
 
     return paras.toArray(new String[paras.size()]);

http://git-wip-us.apache.org/repos/asf/hive/blob/f7dea106/ql/src/java/org/apache/hadoop/hive/ql/processors/ListResourceProcessor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/processors/ListResourceProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/processors/ListResourceProcessor.java
index 7ec36be..91a6aba 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/processors/ListResourceProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/processors/ListResourceProcessor.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hive.ql.processors;
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Schema;
-import org.apache.hadoop.hive.ql.CommandNeedRetryException;
 import org.apache.hadoop.hive.ql.session.SessionState;
 
 import java.util.Arrays;
@@ -44,7 +43,7 @@ public class ListResourceProcessor implements CommandProcessor {
   }
 
   @Override
-  public CommandProcessorResponse run(String command) throws CommandNeedRetryException {
+  public CommandProcessorResponse run(String command) {
     SessionState ss = SessionState.get();
     String[] tokens = command.split("\\s+");
     SessionState.ResourceType t;

http://git-wip-us.apache.org/repos/asf/hive/blob/f7dea106/ql/src/java/org/apache/hadoop/hive/ql/processors/ReloadProcessor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/processors/ReloadProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/processors/ReloadProcessor.java
index b82bd5c..4caab91 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/processors/ReloadProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/processors/ReloadProcessor.java
@@ -22,7 +22,6 @@ import java.io.IOException;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import org.apache.hadoop.hive.ql.CommandNeedRetryException;
 import org.apache.hadoop.hive.ql.session.SessionState;
 
 /**
@@ -32,7 +31,7 @@ public class ReloadProcessor implements CommandProcessor{
   private static final Logger LOG = LoggerFactory.getLogger(ReloadProcessor.class);
 
   @Override
-  public CommandProcessorResponse run(String command) throws CommandNeedRetryException {
+  public CommandProcessorResponse run(String command) {
     SessionState ss = SessionState.get();
     try {
       ss.loadReloadableAuxJars();

http://git-wip-us.apache.org/repos/asf/hive/blob/f7dea106/ql/src/java/org/apache/hadoop/hive/ql/processors/ResetProcessor.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/processors/ResetProcessor.java b/ql/src/java/org/apache/hadoop/hive/ql/processors/ResetProcessor.java
index 144f522..ca39ff9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/processors/ResetProcessor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/processors/ResetProcessor.java
@@ -32,7 +32,6 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveVariableSource;
 import org.apache.hadoop.hive.conf.SystemVariables;
 import org.apache.hadoop.hive.conf.VariableSubstitution;
-import org.apache.hadoop.hive.ql.CommandNeedRetryException;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
 import org.apache.hadoop.hive.ql.session.SessionState;
@@ -43,12 +42,12 @@ public class ResetProcessor implements CommandProcessor {
   private final static String DEFAULT_ARG = "-d";
 
   @Override
-  public CommandProcessorResponse run(String command) throws CommandNeedRetryException {
+  public CommandProcessorResponse run(String command) {
     return run(SessionState.get(), command);
   }
 
   @VisibleForTesting
-  CommandProcessorResponse run(SessionState ss, String command) throws CommandNeedRetryException {
+  CommandProcessorResponse run(SessionState ss, String command) {
     CommandProcessorResponse authErrResp =
         CommandUtil.authorizeCommand(ss, HiveOperationType.RESET, Arrays.asList(command));
     if (authErrResp != null) {

http://git-wip-us.apache.org/repos/asf/hive/blob/f7dea106/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java
index 4508e59..26c6700 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Worker.java
@@ -30,7 +30,6 @@ import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
 import org.apache.hadoop.hive.metastore.api.Table;
 import org.apache.hadoop.hive.metastore.txn.CompactionInfo;
 import org.apache.hadoop.hive.metastore.txn.TxnUtils;
-import org.apache.hadoop.hive.ql.CommandNeedRetryException;
 import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
 import org.apache.hadoop.hive.ql.session.SessionState;
@@ -300,10 +299,6 @@ public class Worker extends CompactorThread {
             (ci.partName == null ? "" : "/" + ci.partName) + " due to: " + cpr);
         }
       }
-      catch(CommandNeedRetryException cnre) {
-        throw new IOException("Could not update stats for table " + ci.getFullTableName() +
-          (ci.partName == null ? "" : "/" + ci.partName) + " due to: " + cnre.getMessage());
-      }
       finally {
         if(localSession != null) {
           localSession.close();

http://git-wip-us.apache.org/repos/asf/hive/blob/f7dea106/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
index d56002d..1e83799 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDTFGetSplits.java
@@ -58,7 +58,6 @@ import org.apache.hadoop.hive.llap.security.LlapTokenIdentifier;
 import org.apache.hadoop.hive.llap.security.LlapTokenLocalClient;
 import org.apache.hadoop.hive.llap.tez.Converters;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
-import org.apache.hadoop.hive.ql.CommandNeedRetryException;
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.ql.QueryPlan;
@@ -269,13 +268,9 @@ public class GenericUDTFGetSplits extends GenericUDTF {
         String ctas = "create temporary table " + tableName + " as " + query;
         LOG.info("Materializing the query for LLAPIF; CTAS: " + ctas);
 
-        try {
-          driver.resetQueryState();
-          HiveConf.setVar(conf, ConfVars.HIVE_EXECUTION_MODE, originalMode);
-          cpr = driver.run(ctas, false);
-        } catch (CommandNeedRetryException e) {
-          throw new HiveException(e);
-        }
+        driver.resetQueryState();
+        HiveConf.setVar(conf, ConfVars.HIVE_EXECUTION_MODE, originalMode);
+        cpr = driver.run(ctas, false);
 
         if(cpr.getResponseCode() != 0) {
           throw new HiveException("Failed to create temp table: " + cpr.getException());

http://git-wip-us.apache.org/repos/asf/hive/blob/f7dea106/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestQueryHooks.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestQueryHooks.java b/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestQueryHooks.java
index 0662875..06a96d5 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestQueryHooks.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/hooks/TestQueryHooks.java
@@ -21,7 +21,6 @@ package org.apache.hadoop.hive.ql.hooks;
 import com.google.common.collect.Lists;
 
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.ql.CommandNeedRetryException;
 import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.ql.session.SessionState;
 
@@ -55,7 +54,7 @@ public class TestQueryHooks {
   }
 
   @Test
-  public void testAllQueryLifeTimeWithParseHooks() throws IllegalAccessException, ClassNotFoundException, InstantiationException, CommandNeedRetryException {
+  public void testAllQueryLifeTimeWithParseHooks() throws Exception {
     String query = "select 1";
     ArgumentMatcher<QueryLifeTimeHookContext> argMatcher = new QueryLifeTimeHookContextMatcher(query);
     QueryLifeTimeHookWithParseHooks mockHook = mock(QueryLifeTimeHookWithParseHooks.class);
@@ -71,7 +70,7 @@ public class TestQueryHooks {
   }
 
   @Test
-  public void testQueryLifeTimeWithParseHooksWithParseError() throws IllegalAccessException, ClassNotFoundException, InstantiationException, CommandNeedRetryException {
+  public void testQueryLifeTimeWithParseHooksWithParseError() throws Exception {
     String query = "invalidquery";
     ArgumentMatcher<QueryLifeTimeHookContext> argMatcher = new QueryLifeTimeHookContextMatcher(query);
     QueryLifeTimeHookWithParseHooks mockHook = mock(QueryLifeTimeHookWithParseHooks.class);
@@ -87,7 +86,7 @@ public class TestQueryHooks {
   }
 
   @Test
-  public void testQueryLifeTimeWithParseHooksWithCompileError() throws IllegalAccessException, ClassNotFoundException, InstantiationException, CommandNeedRetryException {
+  public void testQueryLifeTimeWithParseHooksWithCompileError() throws Exception {
     String query = "select * from foo";
     ArgumentMatcher<QueryLifeTimeHookContext> argMatcher = new QueryLifeTimeHookContextMatcher(query);
     QueryLifeTimeHookWithParseHooks mockHook = mock(QueryLifeTimeHookWithParseHooks.class);
@@ -103,7 +102,7 @@ public class TestQueryHooks {
   }
 
   @Test
-  public void testAllQueryLifeTimeHooks() throws IllegalAccessException, ClassNotFoundException, InstantiationException, CommandNeedRetryException {
+  public void testAllQueryLifeTimeHooks() throws Exception {
     String query = "select 1";
     ArgumentMatcher<QueryLifeTimeHookContext> argMatcher = new QueryLifeTimeHookContextMatcher(query);
     QueryLifeTimeHook mockHook = mock(QueryLifeTimeHook.class);
@@ -117,7 +116,7 @@ public class TestQueryHooks {
   }
 
   @Test
-  public void testQueryLifeTimeWithCompileError() throws IllegalAccessException, ClassNotFoundException, InstantiationException, CommandNeedRetryException {
+  public void testQueryLifeTimeWithCompileError() throws Exception {
     String query = "select * from foo";
     ArgumentMatcher<QueryLifeTimeHookContext> argMatcher = new QueryLifeTimeHookContextMatcher(query);
     QueryLifeTimeHook mockHook = mock(QueryLifeTimeHook.class);

http://git-wip-us.apache.org/repos/asf/hive/blob/f7dea106/ql/src/test/org/apache/hadoop/hive/ql/parse/TestColumnAccess.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestColumnAccess.java b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestColumnAccess.java
index 4da7d92..d2b9327 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/parse/TestColumnAccess.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/parse/TestColumnAccess.java
@@ -23,10 +23,9 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 
-import junit.framework.Assert;
+import org.junit.Assert;
 
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.ql.CommandNeedRetryException;
 import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.ql.QueryPlan;
 import org.apache.hadoop.hive.ql.hooks.ReadEntity;
@@ -38,7 +37,7 @@ import org.junit.Test;
 public class TestColumnAccess {
 
   @BeforeClass
-  public static void Setup() throws CommandNeedRetryException {
+  public static void Setup() throws Exception {
     Driver driver = createDriver();
     int ret = driver.run("create table t1(id1 int, name1 string)").getResponseCode();
     Assert.assertEquals("Checking command success", 0, ret);

http://git-wip-us.apache.org/repos/asf/hive/blob/f7dea106/ql/src/test/org/apache/hadoop/hive/ql/plan/TestReadEntityDirect.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/plan/TestReadEntityDirect.java b/ql/src/test/org/apache/hadoop/hive/ql/plan/TestReadEntityDirect.java
index 4ad821c..88edc12 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/plan/TestReadEntityDirect.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/plan/TestReadEntityDirect.java
@@ -27,7 +27,6 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.ql.CommandNeedRetryException;
 import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.ql.exec.Task;
 import org.apache.hadoop.hive.ql.hooks.ReadEntity;
@@ -49,7 +48,7 @@ import org.junit.Test;
 public class TestReadEntityDirect {
 
   @BeforeClass
-  public static void onetimeSetup() throws CommandNeedRetryException {
+  public static void onetimeSetup() throws Exception {
     Driver driver = createDriver();
     int ret = driver.run("create table t1(i int)").getResponseCode();
     assertEquals("Checking command success", 0, ret);

http://git-wip-us.apache.org/repos/asf/hive/blob/f7dea106/ql/src/test/org/apache/hadoop/hive/ql/processors/TestResetProcessor.java
----------------------------------------------------------------------
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/processors/TestResetProcessor.java b/ql/src/test/org/apache/hadoop/hive/ql/processors/TestResetProcessor.java
index 99f3f6d..26e9083 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/processors/TestResetProcessor.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/processors/TestResetProcessor.java
@@ -22,7 +22,6 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.ql.CommandNeedRetryException;
 import org.apache.hadoop.hive.ql.session.SessionState;
 
 import org.junit.Test;
@@ -35,14 +34,14 @@ import static org.mockito.Mockito.when;
 public class TestResetProcessor {
 
   @Test
-  public void testResetClosesSparkSession() throws CommandNeedRetryException {
+  public void testResetClosesSparkSession() throws Exception {
     SessionState mockSessionState = createMockSparkSessionState();
     new ResetProcessor().run(mockSessionState, "");
     verify(mockSessionState).closeSparkSession();
   }
 
   @Test
-  public void testResetExecutionEngineClosesSparkSession() throws CommandNeedRetryException {
+  public void testResetExecutionEngineClosesSparkSession() throws Exception {
     SessionState mockSessionState = createMockSparkSessionState();
     new ResetProcessor().run(mockSessionState, HiveConf.ConfVars.HIVE_EXECUTION_ENGINE.varname);
     verify(mockSessionState).closeSparkSession();

http://git-wip-us.apache.org/repos/asf/hive/blob/f7dea106/service/src/jamon/org/apache/hive/tmpl/QueryProfileTmpl.jamon
----------------------------------------------------------------------
diff --git a/service/src/jamon/org/apache/hive/tmpl/QueryProfileTmpl.jamon b/service/src/jamon/org/apache/hive/tmpl/QueryProfileTmpl.jamon
index ff7476e..366198e 100644
--- a/service/src/jamon/org/apache/hive/tmpl/QueryProfileTmpl.jamon
+++ b/service/src/jamon/org/apache/hive/tmpl/QueryProfileTmpl.jamon
@@ -179,7 +179,6 @@ org.apache.hadoop.hive.conf.HiveConf;
            <th>End Time</th>
            <th>Elapsed Time (s)</th>
            <th>Requires Lock</th>
-           <th>Retry If Fail</th>
         </tr>
 
        <%if queryInfo.getQueryDisplay() != null && queryInfo.getQueryDisplay().getTaskDisplays() != null %>
@@ -191,7 +190,6 @@ org.apache.hadoop.hive.conf.HiveConf;
                    <td><% taskDisplay.getEndTime() == null ? "" : new Date(taskDisplay.getEndTime()) %></td>
                    <td><% taskDisplay.getElapsedTime() == null ? "" : taskDisplay.getElapsedTime()/1000 %> (s) </td>
                    <td><% taskDisplay.isRequireLock() %></td>
-                   <td><% taskDisplay.isRetryIfFail() %></td>
                </tr>
            </%for>
        </%if>

http://git-wip-us.apache.org/repos/asf/hive/blob/f7dea106/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java b/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
index 9af0e35..5865abe 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/SQLOperation.java
@@ -48,7 +48,6 @@ import org.apache.hadoop.hive.common.metrics.common.MetricsScope;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Schema;
-import org.apache.hadoop.hive.ql.CommandNeedRetryException;
 import org.apache.hadoop.hive.ql.DriverFactory;
 import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.QueryDisplay;
@@ -88,7 +87,6 @@ import org.codehaus.jackson.map.ObjectMapper;
  * SQLOperation.
  *
  */
-@SuppressWarnings("deprecation")
 public class SQLOperation extends ExecuteStatementOperation {
   private IDriver driver = null;
   private CommandProcessorResponse response;
@@ -197,8 +195,6 @@ public class SQLOperation extends ExecuteStatementOperation {
       // In Hive server mode, we are not able to retry in the FetchTask
       // case, when calling fetch queries since execute() has returned.
       // For now, we disable the test attempts.
-      driver.setTryCount(Integer.MAX_VALUE);
-
       response = driver.compileAndRespond(statement);
       if (0 != response.getResponseCode()) {
         throw toSQLException("Error while compiling statement", response);
@@ -249,7 +245,6 @@ public class SQLOperation extends ExecuteStatementOperation {
       // In Hive server mode, we are not able to retry in the FetchTask
       // case, when calling fetch queries since execute() has returned.
       // For now, we disable the test attempts.
-      driver.setTryCount(Integer.MAX_VALUE);
       response = driver.run();
       if (0 != response.getResponseCode()) {
         throw toSQLException("Error while processing statement", response);
@@ -494,8 +489,6 @@ public class SQLOperation extends ExecuteStatementOperation {
       return rowSet;
     } catch (IOException e) {
       throw new HiveSQLException(e);
-    } catch (CommandNeedRetryException e) {
-      throw new HiveSQLException(e);
     } catch (Exception e) {
       throw new HiveSQLException(e);
     } finally {