IMPALA-5903: Inconsistent specification of result set and result set metadata

Before this commit it was quite random which DDL oprations
returned a result set and which didn't.

With this commit, every DDL operations return a summary of
its execution. They declare their result set schema in
Frontend.java, and provide the summary in CalatogOpExecutor.java.

Updated the tests according to the new behavior.

Change-Id: Ic542fb8e49e850052416ac663ee329ee3974e3b9
Reviewed-on: http://gerrit.cloudera.org:8080/9090
Reviewed-by: Alex Behm <alex.b...@cloudera.com>
Tested-by: Impala Public Jenkins <impala-public-jenk...@cloudera.com>


Project: http://git-wip-us.apache.org/repos/asf/impala/repo
Commit: http://git-wip-us.apache.org/repos/asf/impala/commit/2ee914d5
Tree: http://git-wip-us.apache.org/repos/asf/impala/tree/2ee914d5
Diff: http://git-wip-us.apache.org/repos/asf/impala/diff/2ee914d5

Branch: refs/heads/master
Commit: 2ee914d5b365c8230645fdd0604a67eff1edbeb2
Parents: 27c028f
Author: Zoltan Borok-Nagy <borokna...@cloudera.com>
Authored: Thu Apr 5 14:54:27 2018 +0200
Committer: Impala Public Jenkins <impala-public-jenk...@cloudera.com>
Committed: Wed Apr 11 02:21:48 2018 +0000

----------------------------------------------------------------------
 be/src/service/client-request-state.cc          |   7 +-
 .../impala/service/CatalogOpExecutor.java       | 165 ++++++++++++++-----
 .../org/apache/impala/service/Frontend.java     |  25 +--
 shell/impala_client.py                          |   2 +-
 .../queries/QueryTest/alter-table.test          |  29 ++++
 .../queries/QueryTest/chars-tmp-tables.test     |   5 +
 .../QueryTest/compute-stats-incremental.test    |   1 +
 .../queries/QueryTest/compute-stats.test        |  13 +-
 .../queries/QueryTest/create-database.test      |   7 +
 .../QueryTest/create-table-like-file.test       |  14 ++
 .../QueryTest/create-table-like-table.test      |  11 ++
 .../queries/QueryTest/create-table.test         |  15 ++
 .../queries/QueryTest/delimited-latin-text.test |   3 +-
 .../queries/QueryTest/delimited-text.test       |   3 +-
 .../queries/QueryTest/describe-path.test        |   1 +
 .../queries/QueryTest/functions-ddl.test        |  16 ++
 .../queries/QueryTest/grant_revoke.test         |  25 +++
 .../queries/QueryTest/grant_revoke_kudu.test    |   4 +
 .../hbase-compute-stats-incremental.test        |   1 +
 .../queries/QueryTest/insert_bad_expr.test      |   2 +
 .../queries/QueryTest/insert_permutation.test   |   2 +
 .../queries/QueryTest/kudu_alter.test           |  18 ++
 .../queries/QueryTest/kudu_create.test          |  10 +-
 .../queries/QueryTest/kudu_delete.test          |   2 +
 .../queries/QueryTest/kudu_insert.test          |   5 +
 .../queries/QueryTest/kudu_partition_ddl.test   |  10 ++
 .../queries/QueryTest/kudu_stats.test           |   1 +
 .../queries/QueryTest/kudu_update.test          |   1 +
 .../queries/QueryTest/kudu_upsert.test          |   1 +
 .../QueryTest/libs_with_same_filenames.test     |   1 +
 .../queries/QueryTest/load.test                 |   2 +
 .../queries/QueryTest/local-filesystem.test     |   1 +
 .../partition-ddl-predicates-all-fs.test        |   4 +-
 .../QueryTest/test-unmatched-schema.test        |   5 +
 .../queries/QueryTest/truncate-table.test       |   2 +
 .../queries/QueryTest/udf-errors.test           |   7 +
 .../queries/QueryTest/views-ddl.test            |  17 ++
 37 files changed, 366 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/be/src/service/client-request-state.cc
----------------------------------------------------------------------
diff --git a/be/src/service/client-request-state.cc 
b/be/src/service/client-request-state.cc
index 2aedcab..a8b9bfa 100644
--- a/be/src/service/client-request-state.cc
+++ b/be/src/service/client-request-state.cc
@@ -516,6 +516,8 @@ Status ClientRequestState::ExecDdlRequest() {
 
     if (child_queries.size() > 0) {
       RETURN_IF_ERROR(child_query_executor_->ExecAsync(move(child_queries)));
+    } else {
+      SetResultSet({"No partitions selected for incremental stats update."});
     }
     return Status::OK();
   }
@@ -646,6 +648,9 @@ void ClientRequestState::Wait() {
     discard_result(UpdateQueryStatus(status));
   }
   if (status.ok()) {
+    if (stmt_type() == TStmtType::DDL) {
+      DCHECK(catalog_op_type() != TCatalogOpType::DDL || request_result_set_ 
!= nullptr);
+    }
     UpdateNonErrorOperationState(TOperationState::FINISHED_STATE);
   }
   // UpdateQueryStatus() or UpdateNonErrorOperationState() have updated 
operation_state_.
@@ -681,7 +686,7 @@ Status ClientRequestState::WaitInternal() {
 
   if (!returns_result_set()) {
     // Queries that do not return a result are finished at this point. This 
includes
-    // DML operations and a subset of the DDL operations.
+    // DML operations.
     eos_ = true;
   } else if (catalog_op_type() == TCatalogOpType::DDL &&
       ddl_type() == TDdlType::CREATE_TABLE_AS_SELECT) {

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java 
b/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
index 184a839..87513aa 100644
--- a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
+++ b/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
@@ -133,6 +133,7 @@ import org.apache.impala.thrift.TPartitionDef;
 import org.apache.impala.thrift.TPartitionKeyValue;
 import org.apache.impala.thrift.TPartitionStats;
 import org.apache.impala.thrift.TPrivilege;
+import org.apache.impala.thrift.TRangePartitionOperationType;
 import org.apache.impala.thrift.TResetMetadataRequest;
 import org.apache.impala.thrift.TResetMetadataResponse;
 import org.apache.impala.thrift.TResultRow;
@@ -345,6 +346,21 @@ public class CatalogOpExecutor {
   }
 
   /**
+   * Create result set from string 'summary', and attach it to 'response'.
+   */
+  private void addSummary(TDdlExecResponse response, String summary) {
+    TColumnValue resultColVal = new TColumnValue();
+    resultColVal.setString_val(summary);
+    TResultSet resultSet = new TResultSet();
+    resultSet.setSchema(new TResultSetMetadata(Lists.newArrayList(new TColumn(
+        "summary", Type.STRING.toThrift()))));
+    TResultRow resultRow = new TResultRow();
+    resultRow.setColVals(Lists.newArrayList(resultColVal));
+    resultSet.setRows(Lists.newArrayList(resultRow));
+    response.setResult_set(resultSet);
+  }
+
+  /**
    * Execute the ALTER TABLE command according to the TAlterTableParams and 
refresh the
    * table metadata, except for RENAME, ADD PARTITION and DROP PARTITION. This 
call is
    * thread-safe, i.e. concurrent operations on the same table are serialized.
@@ -356,9 +372,6 @@ public class CatalogOpExecutor {
     // When true, loads the table schema and the column stats from the Hive 
Metastore.
     boolean reloadTableSchema = false;
 
-    // When true, sets the result to be reported to the client.
-    boolean setResultSet = false;
-    TColumnValue resultColVal = new TColumnValue();
     Reference<Long> numUpdatedPartitions = new Reference<>(0L);
 
     TableName tableName = TableName.fromThrift(params.getTable_name());
@@ -402,6 +415,11 @@ public class CatalogOpExecutor {
           alterTableAddReplaceCols(tbl, addReplaceColParams.getColumns(),
               addReplaceColParams.isReplace_existing_cols());
           reloadTableSchema = true;
+          if (addReplaceColParams.isReplace_existing_cols()) {
+            addSummary(response, "Table columns have been replaced.");
+          } else {
+            addSummary(response, "New column(s) have been added to the 
table.");
+          }
           break;
         case ADD_PARTITION:
           // Create and add HdfsPartition objects to the corresponding 
HdfsTable and load
@@ -413,17 +431,20 @@ public class CatalogOpExecutor {
             addTableToCatalogUpdate(refreshedTable, response.result);
           }
           reloadMetadata = false;
+          addSummary(response, "New partition has been added to the table.");
           break;
         case DROP_COLUMN:
           TAlterTableDropColParams dropColParams = params.getDrop_col_params();
           alterTableDropCol(tbl, dropColParams.getCol_name());
           reloadTableSchema = true;
+          addSummary(response, "Column has been dropped.");
           break;
         case ALTER_COLUMN:
           TAlterTableAlterColParams alterColParams = 
params.getAlter_col_params();
           alterTableAlterCol(tbl, alterColParams.getCol_name(),
               alterColParams.getNew_col_def());
           reloadTableSchema = true;
+          addSummary(response, "Column has been altered.");
           break;
         case DROP_PARTITION:
           TAlterTableDropPartitionParams dropPartParams =
@@ -440,9 +461,8 @@ public class CatalogOpExecutor {
             refreshedTable.setCatalogVersion(newCatalogVersion);
             addTableToCatalogUpdate(refreshedTable, response.result);
           }
-          resultColVal.setString_val(
+          addSummary(response,
               "Dropped " + numUpdatedPartitions.getRef() + " partition(s).");
-          setResultSet = true;
           reloadMetadata = false;
           break;
         case RENAME_TABLE:
@@ -458,12 +478,11 @@ public class CatalogOpExecutor {
               fileFormatParams.getFile_format(), numUpdatedPartitions);
 
           if (fileFormatParams.isSetPartition_set()) {
-            resultColVal.setString_val(
+            addSummary(response,
                 "Updated " + numUpdatedPartitions.getRef() + " partition(s).");
           } else {
-            resultColVal.setString_val("Updated table.");
+            addSummary(response, "Updated table.");
           }
-          setResultSet = true;
           break;
         case SET_ROW_FORMAT:
           TAlterTableSetRowFormatParams rowFormatParams =
@@ -472,29 +491,33 @@ public class CatalogOpExecutor {
               rowFormatParams.getPartition_set(), 
rowFormatParams.getRow_format(),
               numUpdatedPartitions);
           if (rowFormatParams.isSetPartition_set()) {
-            resultColVal.setString_val(
+            addSummary(response,
                 "Updated " + numUpdatedPartitions.getRef() + " partition(s).");
           } else {
-            resultColVal.setString_val("Updated table.");
+            addSummary(response, "Updated table.");
           }
-          setResultSet = true;
           break;
         case SET_LOCATION:
           TAlterTableSetLocationParams setLocationParams =
               params.getSet_location_params();
-          reloadFileMetadata = alterTableSetLocation(tbl,
-              setLocationParams.getPartition_spec(), 
setLocationParams.getLocation());
+          List<TPartitionKeyValue> partitionSpec = 
setLocationParams.getPartition_spec();
+          reloadFileMetadata = alterTableSetLocation(tbl, partitionSpec,
+              setLocationParams.getLocation());
+          if (partitionSpec == null) {
+            addSummary(response, "New location has been set.");
+          } else {
+            addSummary(response, "New location has been set for the specified 
partition.");
+          }
           break;
         case SET_TBL_PROPERTIES:
           alterTableSetTblProperties(tbl, 
params.getSet_tbl_properties_params(),
               numUpdatedPartitions);
           if (params.getSet_tbl_properties_params().isSetPartition_set()) {
-            resultColVal.setString_val(
+            addSummary(response,
                 "Updated " + numUpdatedPartitions.getRef() + " partition(s).");
           } else {
-            resultColVal.setString_val("Updated table.");
+            addSummary(response, "Updated table.");
           }
-          setResultSet = true;
           break;
         case UPDATE_STATS:
           Preconditions.checkState(params.isSetUpdate_stats_params());
@@ -502,9 +525,8 @@ public class CatalogOpExecutor {
           alterTableUpdateStats(tbl, params.getUpdate_stats_params(),
               numUpdatedPartitions, numUpdatedColumns);
           reloadTableSchema = true;
-          resultColVal.setString_val("Updated " + 
numUpdatedPartitions.getRef() +
+          addSummary(response, "Updated " + numUpdatedPartitions.getRef() +
               " partition(s) and " + numUpdatedColumns.getRef() + " 
column(s).");
-          setResultSet = true;
           break;
         case SET_CACHED:
           Preconditions.checkState(params.isSetSet_cached_params());
@@ -513,17 +535,17 @@ public class CatalogOpExecutor {
           if (params.getSet_cached_params().getPartition_set() == null) {
             reloadFileMetadata =
                 alterTableSetCached(tbl, params.getSet_cached_params());
-            resultColVal.setString_val(op + "table.");
+            addSummary(response, op + "table.");
           } else {
             alterPartitionSetCached(tbl, params.getSet_cached_params(),
                 numUpdatedPartitions);
-            resultColVal.setString_val(
+            addSummary(response,
                 op + numUpdatedPartitions.getRef() + " partition(s).");
           }
-          setResultSet = true;
           break;
         case RECOVER_PARTITIONS:
           alterTableRecoverPartitions(tbl);
+          addSummary(response, "Partitions have been recovered.");
           break;
         default:
           throw new UnsupportedOperationException(
@@ -535,16 +557,6 @@ public class CatalogOpExecutor {
             reloadTableSchema, null);
         addTableToCatalogUpdate(tbl, response.result);
       }
-
-      if (setResultSet) {
-        TResultSet resultSet = new TResultSet();
-        resultSet.setSchema(new TResultSetMetadata(Lists.newArrayList(
-            new TColumn("summary", Type.STRING.toThrift()))));
-        TResultRow resultRow = new TResultRow();
-        resultRow.setColVals(Lists.newArrayList(resultColVal));
-        resultSet.setRows(Lists.newArrayList(resultRow));
-        response.setResult_set(resultSet);
-      }
     } finally {
       context.stop();
       
Preconditions.checkState(!catalog_.getLock().isWriteLockedByCurrentThread());
@@ -575,21 +587,27 @@ public class CatalogOpExecutor {
             params.getAdd_replace_cols_params();
         KuduCatalogOpExecutor.addColumn((KuduTable) tbl,
             addReplaceColParams.getColumns());
+        addSummary(response, "Column has been added/replaced.");
         break;
       case DROP_COLUMN:
         TAlterTableDropColParams dropColParams = params.getDrop_col_params();
         KuduCatalogOpExecutor.dropColumn((KuduTable) tbl,
             dropColParams.getCol_name());
+        addSummary(response, "Column has been dropped.");
         break;
       case ALTER_COLUMN:
         TAlterTableAlterColParams alterColParams = 
params.getAlter_col_params();
         KuduCatalogOpExecutor.alterColumn((KuduTable) tbl, 
alterColParams.getCol_name(),
             alterColParams.getNew_col_def());
+        addSummary(response, "Column has been altered.");
         break;
       case ADD_DROP_RANGE_PARTITION:
         TAlterTableAddDropRangePartitionParams partParams =
             params.getAdd_drop_range_partition_params();
         KuduCatalogOpExecutor.addDropRangePartition((KuduTable) tbl, 
partParams);
+        addSummary(response, "Range partition has been " +
+            (partParams.type == TRangePartitionOperationType.ADD ?
+            "added." : "dropped."));
         break;
       default:
         throw new UnsupportedOperationException(
@@ -692,6 +710,7 @@ public class CatalogOpExecutor {
       try (MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
         tbl.load(true, msClient.getHiveClient(), msTbl);
       }
+      addSummary(resp, "View has been altered.");
       tbl.setCatalogVersion(newCatalogVersion);
       addTableToCatalogUpdate(tbl, resp.result);
     } finally {
@@ -946,6 +965,7 @@ public class CatalogOpExecutor {
       Preconditions.checkNotNull(existingDb);
       
resp.getResult().addToUpdated_catalog_objects(existingDb.toTCatalogObject());
       resp.getResult().setVersion(existingDb.getCatalogVersion());
+      addSummary(resp, "Database already exists.");
       return;
     }
     org.apache.hadoop.hive.metastore.api.Database db =
@@ -966,11 +986,13 @@ public class CatalogOpExecutor {
         try {
           msClient.getHiveClient().createDatabase(db);
           newDb = catalog_.addDb(dbName, db);
+          addSummary(resp, "Database has been created.");
         } catch (AlreadyExistsException e) {
           if (!params.if_not_exists) {
             throw new ImpalaRuntimeException(
                 String.format(HMS_RPC_ERROR_FORMAT_STR, "createDatabase"), e);
           }
+          addSummary(resp, "Database already exists.");
           if (LOG.isTraceEnabled()) {
             LOG.trace(String.format("Ignoring '%s' when creating database %s 
because " +
                 "IF NOT EXISTS was specified.", e, dbName));
@@ -1022,6 +1044,7 @@ public class CatalogOpExecutor {
             throw new CatalogException("Function " + fn.functionName() +
                 " already exists.");
           }
+          addSummary(resp, "Function already exists.");
           return;
         }
       }
@@ -1059,6 +1082,9 @@ public class CatalogOpExecutor {
       if (!addedFunctions.isEmpty()) {
         resp.result.setUpdated_catalog_objects(addedFunctions);
         resp.result.setVersion(catalog_.getCatalogVersion());
+        addSummary(resp, "Function has been created.");
+      } else {
+        addSummary(resp, "Function already exists.");
       }
     }
   }
@@ -1073,6 +1099,7 @@ public class CatalogOpExecutor {
         throw new ImpalaRuntimeException("Data source " + dataSource.getName() 
+
             " already exists.");
       }
+      addSummary(resp, "Data source already exists.");
       
resp.result.addToUpdated_catalog_objects(existingDataSource.toTCatalogObject());
       resp.result.setVersion(existingDataSource.getCatalogVersion());
       return;
@@ -1080,6 +1107,7 @@ public class CatalogOpExecutor {
     catalog_.addDataSource(dataSource);
     resp.result.addToUpdated_catalog_objects(dataSource.toTCatalogObject());
     resp.result.setVersion(dataSource.getCatalogVersion());
+    addSummary(resp, "Data source has been created.");
   }
 
   private void dropDataSource(TDropDataSourceParams params, TDdlExecResponse 
resp)
@@ -1091,12 +1119,14 @@ public class CatalogOpExecutor {
         throw new ImpalaRuntimeException("Data source " + 
params.getData_source() +
             " does not exists.");
       }
+      addSummary(resp, "Data source does not exist.");
       // No data source was removed.
       resp.result.setVersion(catalog_.getCatalogVersion());
       return;
     }
     resp.result.addToRemoved_catalog_objects(dataSource.toTCatalogObject());
     resp.result.setVersion(dataSource.getCatalogVersion());
+    addSummary(resp, "Data source has been dropped.");
   }
 
   /**
@@ -1126,7 +1156,10 @@ public class CatalogOpExecutor {
         HdfsTable hdfsTbl = (HdfsTable) table;
         List<HdfsPartition> partitions =
             hdfsTbl.getPartitionsFromPartitionSet(params.getPartition_set());
-        if (partitions.isEmpty()) return;
+        if (partitions.isEmpty()) {
+          addSummary(resp, "No partitions found for table.");
+          return;
+        }
 
         for(HdfsPartition partition : partitions) {
           if (partition.getPartitionStats() != null) {
@@ -1141,6 +1174,7 @@ public class CatalogOpExecutor {
       }
       loadTableMetadata(table, newCatalogVersion, false, true, null);
       addTableToCatalogUpdate(table, resp.result);
+      addSummary(resp, "Stats have been dropped.");
     } finally {
       
Preconditions.checkState(!catalog_.getLock().isWriteLockedByCurrentThread());
       table.getLock().unlock();
@@ -1258,11 +1292,13 @@ public class CatalogOpExecutor {
       try (MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
         msClient.getHiveClient().dropDatabase(
             params.getDb(), true, params.if_exists, params.cascade);
+        addSummary(resp, "Database has been dropped.");
       } catch (TException e) {
         throw new ImpalaRuntimeException(
             String.format(HMS_RPC_ERROR_FORMAT_STR, "dropDatabase"), e);
       }
       Db removedDb = catalog_.removeDb(params.getDb());
+
       if (removedDb == null) {
         // Nothing was removed from the catalogd's cache.
         resp.result.setVersion(catalog_.getCatalogVersion());
@@ -1277,6 +1313,7 @@ public class CatalogOpExecutor {
     Preconditions.checkNotNull(removedObject);
     resp.result.setVersion(removedObject.getCatalog_version());
     resp.result.addToRemoved_catalog_objects(removedObject);
+    addSummary(resp, "Database has been dropped.");
   }
 
   /**
@@ -1333,14 +1370,20 @@ public class CatalogOpExecutor {
     synchronized (metastoreDdlLock_) {
       Db db = catalog_.getDb(params.getTable_name().db_name);
       if (db == null) {
-        if (params.if_exists) return;
-        throw new CatalogException("Database does not exist: " +
-            params.getTable_name().db_name);
+        String dbNotExist = "Database does not exist: " + 
params.getTable_name().db_name;
+        if (params.if_exists) {
+          addSummary(resp, dbNotExist);
+          return;
+        }
+        throw new CatalogException(dbNotExist);
       }
       Table existingTbl = db.getTable(params.getTable_name().table_name);
       if (existingTbl == null) {
-        if (params.if_exists) return;
-        throw new CatalogException("Table/View does not exist: " + tableName);
+        if (params.if_exists) {
+          addSummary(resp, (params.is_table ? "Table " : "View ") + "does not 
exist.");
+          return;
+        }
+        throw new CatalogException("Table/View does not exist.");
       }
 
       // Retrieve the HMS table to determine if this is a Kudu table.
@@ -1366,9 +1409,13 @@ public class CatalogOpExecutor {
       // fixed.
       if (params.isSetIs_table() && ((params.is_table && existingTbl 
instanceof View)
           || (!params.is_table && !(existingTbl instanceof View)))) {
-        if (params.if_exists) return;
         String errorMsg = "DROP " + (params.is_table ? "TABLE " : "VIEW ") +
             "not allowed on a " + (params.is_table ? "view: " : "table: ") + 
tableName;
+        if (params.if_exists) {
+          addSummary(resp, "Drop " + (params.is_table ? "table " : "view ") +
+              "is not allowed on a " + (params.is_table ? "view." : "table."));
+          return;
+        }
         throw new CatalogException(errorMsg);
       }
       try (MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
@@ -1382,6 +1429,7 @@ public class CatalogOpExecutor {
         throw new ImpalaRuntimeException(
             String.format(HMS_RPC_ERROR_FORMAT_STR, "dropTable"), e);
       }
+      addSummary(resp, (params.is_table ? "Table " : "View ") + "has been 
dropped.");
 
       Table table = catalog_.removeTable(params.getTable_name().db_name,
           params.getTable_name().table_name);
@@ -1442,7 +1490,10 @@ public class CatalogOpExecutor {
     try {
       table = getExistingTable(tblName.getDb_name(), tblName.getTable_name());
     } catch (TableNotFoundException e) {
-      if (params.if_exists) return;
+      if (params.if_exists) {
+        addSummary(resp, "Table does not exist.");
+        return;
+      }
       throw e;
     }
     Preconditions.checkNotNull(table);
@@ -1472,6 +1523,7 @@ public class CatalogOpExecutor {
         throw new CatalogException(String.format("Failed to truncate table: 
%s.\n" +
             "Table may be in a partially truncated state.", fqName), e);
       }
+      addSummary(resp, "Table has been truncated.");
 
       loadTableMetadata(table, newCatalogVersion, true, true, null);
       addTableToCatalogUpdate(table, resp.result);
@@ -1491,6 +1543,7 @@ public class CatalogOpExecutor {
             throw new CatalogException("Database: " + fName.getDb()
                 + " does not exist.");
         }
+        addSummary(resp, "Database does not exist.");
         return;
       }
       List<TCatalogObject> removedFunctions = Lists.newArrayList();
@@ -1524,7 +1577,10 @@ public class CatalogOpExecutor {
       }
 
       if (!removedFunctions.isEmpty()) {
+        addSummary(resp, "Function has been dropped.");
         resp.result.setRemoved_catalog_objects(removedFunctions);
+      } else {
+        addSummary(resp, "Function does not exist.");
       }
       resp.result.setVersion(catalog_.getCatalogVersion());
     }
@@ -1546,6 +1602,7 @@ public class CatalogOpExecutor {
 
     Table existingTbl = catalog_.getTableNoThrow(tableName.getDb(), 
tableName.getTbl());
     if (params.if_not_exists && existingTbl != null) {
+      addSummary(response, "Table already exists.");
       LOG.trace(String.format("Skipping table creation because %s already 
exists and " +
           "IF NOT EXISTS was specified.", tableName));
       existingTbl.getLock().lock();
@@ -1667,10 +1724,14 @@ public class CatalogOpExecutor {
             " dropped. The log contains more information.", 
newTable.getTableName(),
             kuduTableName), e);
       }
-      if (e instanceof AlreadyExistsException && params.if_not_exists) return 
false;
+      if (e instanceof AlreadyExistsException && params.if_not_exists) {
+        addSummary(response, "Table already exists.");
+        return false;
+      }
       throw new ImpalaRuntimeException(
           String.format(HMS_RPC_ERROR_FORMAT_STR, "createTable"), e);
     }
+    addSummary(response, "Table has been created.");
     return true;
   }
 
@@ -1687,6 +1748,7 @@ public class CatalogOpExecutor {
     synchronized (metastoreDdlLock_) {
       try (MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
         msClient.getHiveClient().createTable(newTable);
+        addSummary(response, "Table has been created.");
         // If this table should be cached, and the table location was not 
specified by
         // the user, an extra step is needed to read the table to find the 
location.
         if (cacheOp != null && cacheOp.isSet_cached() &&
@@ -1695,7 +1757,10 @@ public class CatalogOpExecutor {
               newTable.getDbName(), newTable.getTableName());
         }
       } catch (Exception e) {
-        if (e instanceof AlreadyExistsException && if_not_exists) return false;
+        if (e instanceof AlreadyExistsException && if_not_exists) {
+          addSummary(response, "Table already exists");
+          return false;
+        }
         throw new ImpalaRuntimeException(
             String.format(HMS_RPC_ERROR_FORMAT_STR, "createTable"), e);
       }
@@ -1739,7 +1804,11 @@ public class CatalogOpExecutor {
         new org.apache.hadoop.hive.metastore.api.Table();
     setViewAttributes(params, view);
     LOG.trace(String.format("Creating view %s", tableName));
-    createTable(view, params.if_not_exists, null, response);
+    if (!createTable(view, params.if_not_exists, null, response)) {
+      addSummary(response, "View already exists.");
+    } else {
+      addSummary(response, "View has been created.");
+    }
   }
 
   /**
@@ -1762,6 +1831,7 @@ public class CatalogOpExecutor {
 
     Table existingTbl = catalog_.getTableNoThrow(tblName.getDb(), 
tblName.getTbl());
     if (params.if_not_exists && existingTbl != null) {
+      addSummary(response, "Table already exists.");
       LOG.trace(String.format("Skipping table creation because %s already 
exists and " +
           "IF NOT EXISTS was specified.", tblName));
       existingTbl.getLock().lock();
@@ -2226,6 +2296,7 @@ public class CatalogOpExecutor {
     
response.result.addToRemoved_catalog_objects(result.first.toMinimalTCatalogObject());
     
response.result.addToUpdated_catalog_objects(result.second.toTCatalogObject());
     response.result.setVersion(result.second.getCatalogVersion());
+    addSummary(response, "Renaming was successful.");
   }
 
   /**
@@ -2838,6 +2909,7 @@ public class CatalogOpExecutor {
       if (role == null) {
         // Nothing was removed from the catalogd's cache.
         resp.result.setVersion(catalog_.getCatalogVersion());
+        addSummary(resp, "No such role.");
         return;
       }
     } else {
@@ -2852,8 +2924,10 @@ public class CatalogOpExecutor {
     catalogObject.setCatalog_version(role.getCatalogVersion());
     if (createDropRoleParams.isIs_drop()) {
       resp.result.addToRemoved_catalog_objects(catalogObject);
+      addSummary(resp, "Role has been dropped.");
     } else {
       resp.result.addToUpdated_catalog_objects(catalogObject);
+      addSummary(resp, "Role has been created.");
     }
     resp.result.setVersion(role.getCatalogVersion());
   }
@@ -2884,6 +2958,11 @@ public class CatalogOpExecutor {
     catalogObject.setRole(role.toThrift());
     catalogObject.setCatalog_version(role.getCatalogVersion());
     resp.result.addToUpdated_catalog_objects(catalogObject);
+    if (grantRevokeRoleParams.isIs_grant()) {
+      addSummary(resp, "Role has been granted.");
+    } else {
+      addSummary(resp, "Role has been revoked.");
+    }
     resp.result.setVersion(role.getCatalogVersion());
   }
 
@@ -2902,9 +2981,11 @@ public class CatalogOpExecutor {
     if (grantRevokePrivParams.isIs_grant()) {
       rolePrivileges = 
catalog_.getSentryProxy().grantRolePrivileges(requestingUser,
           roleName, privileges);
+      addSummary(resp, "Privilege(s) have been granted.");
     } else {
       rolePrivileges = 
catalog_.getSentryProxy().revokeRolePrivileges(requestingUser,
           roleName, privileges, grantRevokePrivParams.isHas_grant_opt());
+      addSummary(resp, "Privilege(s) have been revoked.");
     }
     Preconditions.checkNotNull(rolePrivileges);
     List<TCatalogObject> updatedPrivs = Lists.newArrayList();

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/fe/src/main/java/org/apache/impala/service/Frontend.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/service/Frontend.java 
b/fe/src/main/java/org/apache/impala/service/Frontend.java
index 539fe31..392c249 100644
--- a/fe/src/main/java/org/apache/impala/service/Frontend.java
+++ b/fe/src/main/java/org/apache/impala/service/Frontend.java
@@ -334,21 +334,18 @@ public class Frontend {
       req.setDdl_type(TDdlType.ALTER_TABLE);
       req.setAlter_table_params(analysis.getAlterTableStmt().toThrift());
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else if (analysis.isAlterViewStmt()) {
       ddl.op_type = TCatalogOpType.DDL;
       TDdlExecRequest req = new TDdlExecRequest();
       req.setDdl_type(TDdlType.ALTER_VIEW);
       req.setAlter_view_params(analysis.getAlterViewStmt().toThrift());
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else if (analysis.isCreateTableStmt()) {
       ddl.op_type = TCatalogOpType.DDL;
       TDdlExecRequest req = new TDdlExecRequest();
       req.setDdl_type(TDdlType.CREATE_TABLE);
       req.setCreate_table_params(analysis.getCreateTableStmt().toThrift());
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else if (analysis.isCreateTableAsSelectStmt()) {
       ddl.op_type = TCatalogOpType.DDL;
       TDdlExecRequest req = new TDdlExecRequest();
@@ -356,29 +353,24 @@ public class Frontend {
       req.setCreate_table_params(
           analysis.getCreateTableAsSelectStmt().getCreateStmt().toThrift());
       ddl.setDdl_params(req);
-      metadata.setColumns(Arrays.asList(
-          new TColumn("summary", Type.STRING.toThrift())));
     } else if (analysis.isCreateTableLikeStmt()) {
       ddl.op_type = TCatalogOpType.DDL;
       TDdlExecRequest req = new TDdlExecRequest();
       req.setDdl_type(TDdlType.CREATE_TABLE_LIKE);
       
req.setCreate_table_like_params(analysis.getCreateTableLikeStmt().toThrift());
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else if (analysis.isCreateViewStmt()) {
       ddl.op_type = TCatalogOpType.DDL;
       TDdlExecRequest req = new TDdlExecRequest();
       req.setDdl_type(TDdlType.CREATE_VIEW);
       req.setCreate_view_params(analysis.getCreateViewStmt().toThrift());
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else if (analysis.isCreateDbStmt()) {
       ddl.op_type = TCatalogOpType.DDL;
       TDdlExecRequest req = new TDdlExecRequest();
       req.setDdl_type(TDdlType.CREATE_DATABASE);
       req.setCreate_db_params(analysis.getCreateDbStmt().toThrift());
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else if (analysis.isCreateUdfStmt()) {
       ddl.op_type = TCatalogOpType.DDL;
       CreateUdfStmt stmt = (CreateUdfStmt) analysis.getStmt();
@@ -386,7 +378,6 @@ public class Frontend {
       req.setDdl_type(TDdlType.CREATE_FUNCTION);
       req.setCreate_fn_params(stmt.toThrift());
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else if (analysis.isCreateUdaStmt()) {
       ddl.op_type = TCatalogOpType.DDL;
       TDdlExecRequest req = new TDdlExecRequest();
@@ -394,7 +385,6 @@ public class Frontend {
       CreateUdaStmt stmt = (CreateUdaStmt)analysis.getStmt();
       req.setCreate_fn_params(stmt.toThrift());
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else if (analysis.isCreateDataSrcStmt()) {
       ddl.op_type = TCatalogOpType.DDL;
       TDdlExecRequest req = new TDdlExecRequest();
@@ -402,21 +392,18 @@ public class Frontend {
       CreateDataSrcStmt stmt = (CreateDataSrcStmt)analysis.getStmt();
       req.setCreate_data_source_params(stmt.toThrift());
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else if (analysis.isComputeStatsStmt()) {
       ddl.op_type = TCatalogOpType.DDL;
       TDdlExecRequest req = new TDdlExecRequest();
       req.setDdl_type(TDdlType.COMPUTE_STATS);
       req.setCompute_stats_params(analysis.getComputeStatsStmt().toThrift());
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else if (analysis.isDropDbStmt()) {
       ddl.op_type = TCatalogOpType.DDL;
       TDdlExecRequest req = new TDdlExecRequest();
       req.setDdl_type(TDdlType.DROP_DATABASE);
       req.setDrop_db_params(analysis.getDropDbStmt().toThrift());
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else if (analysis.isDropTableOrViewStmt()) {
       ddl.op_type = TCatalogOpType.DDL;
       TDdlExecRequest req = new TDdlExecRequest();
@@ -424,7 +411,6 @@ public class Frontend {
       req.setDdl_type(stmt.isDropTable() ? TDdlType.DROP_TABLE : 
TDdlType.DROP_VIEW);
       req.setDrop_table_or_view_params(stmt.toThrift());
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else if (analysis.isTruncateStmt()) {
       ddl.op_type = TCatalogOpType.DDL;
       TDdlExecRequest req = new TDdlExecRequest();
@@ -432,7 +418,6 @@ public class Frontend {
       req.setDdl_type(TDdlType.TRUNCATE_TABLE);
       req.setTruncate_params(stmt.toThrift());
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else if (analysis.isDropFunctionStmt()) {
       ddl.op_type = TCatalogOpType.DDL;
       TDdlExecRequest req = new TDdlExecRequest();
@@ -440,7 +425,6 @@ public class Frontend {
       DropFunctionStmt stmt = (DropFunctionStmt)analysis.getStmt();
       req.setDrop_fn_params(stmt.toThrift());
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else if (analysis.isDropDataSrcStmt()) {
       ddl.op_type = TCatalogOpType.DDL;
       TDdlExecRequest req = new TDdlExecRequest();
@@ -448,7 +432,6 @@ public class Frontend {
       DropDataSrcStmt stmt = (DropDataSrcStmt)analysis.getStmt();
       req.setDrop_data_source_params(stmt.toThrift());
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else if (analysis.isDropStatsStmt()) {
       ddl.op_type = TCatalogOpType.DDL;
       TDdlExecRequest req = new TDdlExecRequest();
@@ -456,7 +439,6 @@ public class Frontend {
       DropStatsStmt stmt = (DropStatsStmt) analysis.getStmt();
       req.setDrop_stats_params(stmt.toThrift());
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else if (analysis.isResetMetadataStmt()) {
       ddl.op_type = TCatalogOpType.RESET_METADATA;
       ResetMetadataStmt resetMetadataStmt = (ResetMetadataStmt) 
analysis.getStmt();
@@ -499,7 +481,6 @@ public class Frontend {
       req.setCreate_drop_role_params(params);
       ddl.op_type = TCatalogOpType.DDL;
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else if (analysis.isGrantRevokeRoleStmt()) {
       GrantRevokeRoleStmt grantRoleStmt = (GrantRevokeRoleStmt) 
analysis.getStmt();
       TGrantRevokeRoleParams params = grantRoleStmt.toThrift();
@@ -508,7 +489,6 @@ public class Frontend {
       req.setGrant_revoke_role_params(params);
       ddl.op_type = TCatalogOpType.DDL;
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else if (analysis.isGrantRevokePrivStmt()) {
       GrantRevokePrivStmt grantRevokePrivStmt = (GrantRevokePrivStmt) 
analysis.getStmt();
       TGrantRevokePrivParams params = grantRevokePrivStmt.toThrift();
@@ -518,10 +498,13 @@ public class Frontend {
       req.setGrant_revoke_priv_params(params);
       ddl.op_type = TCatalogOpType.DDL;
       ddl.setDdl_params(req);
-      metadata.setColumns(Collections.<TColumn>emptyList());
     } else {
       throw new IllegalStateException("Unexpected CatalogOp statement type.");
     }
+    // All DDL commands return a string summarizing the outcome of the DDL.
+    if (ddl.op_type == TCatalogOpType.DDL) {
+      metadata.setColumns(Arrays.asList(new TColumn("summary", 
Type.STRING.toThrift())));
+    }
     result.setResult_set_metadata(metadata);
     ddl.setSync_ddl(result.getQuery_options().isSync_ddl());
     result.setCatalog_op_request(ddl);

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/shell/impala_client.py
----------------------------------------------------------------------
diff --git a/shell/impala_client.py b/shell/impala_client.py
index d4bfbee..5fa50b0 100755
--- a/shell/impala_client.py
+++ b/shell/impala_client.py
@@ -486,7 +486,7 @@ class ImpalaClient(object):
 
   def expect_result_metadata(self, query_str):
     """ Given a query string, return True if impalad expects result metadata"""
-    excluded_query_types = ['use', 'drop']
+    excluded_query_types = ['use']
     if True in set(map(query_str.startswith, excluded_query_types)):
       return False
     return True

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/alter-table.test
----------------------------------------------------------------------
diff --git 
a/testdata/workloads/functional-query/queries/QueryTest/alter-table.test 
b/testdata/workloads/functional-query/queries/QueryTest/alter-table.test
index 555a599..013b8a9 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/alter-table.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/alter-table.test
@@ -4,6 +4,7 @@
 create external table t1 (i int)
 location '$FILESYSTEM_PREFIX/test-warehouse/$DATABASE.db/t1_tmp1'
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 describe t1
@@ -16,6 +17,9 @@ string,string,string
 # Add some columns
 alter table t1 add columns (t tinyint, s string comment 'Str Col')
 ---- RESULTS
+'New column(s) have been added to the table.'
+---- TYPES
+string
 ====
 ---- QUERY
 describe t1
@@ -29,6 +33,7 @@ string,string,string
 ---- QUERY
 alter table t1 rename to t2
 ---- RESULTS
+'Renaming was successful.'
 ====
 ---- QUERY
 show tables
@@ -42,6 +47,7 @@ string
 # Move the table to a different database
 alter table t2 rename to $DATABASE2.t1_inotherdb
 ---- RESULTS
+'Renaming was successful.'
 ====
 ---- QUERY
 # No longer appears in this database
@@ -63,6 +69,7 @@ string
 # Move the table back to this database
 alter table $DATABASE2.t1_inotherdb rename to t2
 ---- RESULTS
+'Renaming was successful.'
 ====
 ---- QUERY
 # make sure the new table shows the same columns as the old table
@@ -77,6 +84,7 @@ string,string,string
 ---- QUERY
 alter table t2 drop column t
 ---- RESULTS
+'Column has been dropped.'
 ====
 ---- QUERY
 # The dropped column no longer shows up
@@ -91,6 +99,7 @@ string,string,string
 # Replace the columns with new values
 alter table t2 replace columns (c1 bigint comment 'id col', c2 string, c3 int)
 ---- RESULTS
+'Table columns have been replaced.'
 ====
 ---- QUERY
 describe t2
@@ -118,14 +127,17 @@ bigint,string,int
 ---- QUERY
 alter table t2 change column c2 int_col int comment 'changed to int col'
 ---- RESULTS
+'Column has been altered.'
 ====
 ---- QUERY
 alter table t2 change column c1 id_col bigint
 ---- RESULTS
+'Column has been altered.'
 ====
 ---- QUERY
 alter table t2 change column c3 c3 int comment 'added a comment'
 ---- RESULTS
+'Column has been altered.'
 ====
 ---- QUERY
 describe t2
@@ -151,6 +163,7 @@ x array<int>,
 y map<string,float> comment 'Map Col',
 z struct<f1:boolean,f2:bigint>)
 ---- RESULTS
+'New column(s) have been added to the table.'
 ====
 ---- QUERY
 describe t2
@@ -171,6 +184,7 @@ a int comment 'Int Col',
 b struct<f1:array<int>,f2:map<string,struct<f1:bigint>>>,
 c double)
 ---- RESULTS
+'Table columns have been replaced.'
 ====
 ---- QUERY
 describe t2
@@ -217,6 +231,7 @@ string,string,string
 ---- QUERY
 create external table jointbl_test like functional.jointbl
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # Expect new table to be empty
@@ -282,6 +297,7 @@ alter table t_part add if not exists partition (j=1, 
s='2012');
 alter table t_part add if not exists partition (j=1, s='2012/withslash');
 alter table t_part add partition (j=1, s=substring('foo2013bar', 4, 8));
 ---- RESULTS
+'New partition has been added to the table.'
 ====
 ---- QUERY
 # Add another partition that points to the same location as another partition.
@@ -289,6 +305,7 @@ alter table t_part add partition (j=1, 
s=substring('foo2013bar', 4, 8));
 alter table t_part add partition (j=100, s='same_location')
 location '$FILESYSTEM_PREFIX/test-warehouse/$DATABASE.db/t_part_tmp/j=1/s=2012'
 ---- RESULTS
+'New partition has been added to the table.'
 ====
 ---- QUERY
 # Add another partition that points to an existing data location that does not
@@ -296,6 +313,7 @@ location 
'$FILESYSTEM_PREFIX/test-warehouse/$DATABASE.db/t_part_tmp/j=1/s=2012'
 alter table t_part add partition (j=101, s='different_part_dir')
 location '$FILESYSTEM_PREFIX/test-warehouse/$DATABASE.db/part_data/'
 ---- RESULTS
+'New partition has been added to the table.'
 ====
 ---- QUERY
 insert overwrite table t_part partition(j=1, s='2012') select 2 from 
functional.alltypes limit 2
@@ -337,10 +355,12 @@ int,int,string
 ---- QUERY
 alter table t_part add partition (j=NULL, s='2013')
 ---- RESULTS
+'New partition has been added to the table.'
 ====
 ---- QUERY
 alter table t_part add partition (j=NULL, s=NULL)
 ---- RESULTS
+'New partition has been added to the table.'
 ====
 ---- QUERY
 # Drop the partition that points to a duplication location. The data will no 
longer
@@ -527,6 +547,7 @@ int,int,string
 # rename a partitioned table
 alter table t_part rename to t_part2
 ---- RESULTS
+'Renaming was successful.'
 ====
 ---- QUERY
 # only the new table shows up
@@ -558,6 +579,7 @@ int
 alter table alltypes_test partition(month=4, year=2009)
 set location 
'$FILESYSTEM_PREFIX/test-warehouse/alltypes_seq_snap/year=2009/month=4'
 ---- RESULTS
+'New location has been set for the specified partition.'
 ====
 ---- QUERY
 alter table alltypes_test partition(month=4, year=2009)
@@ -590,6 +612,7 @@ int,bigint
 alter table alltypes_test partition(month=cast(1+4 as int), year=cast(100*20+9 
as int))
 set location '$FILESYSTEM_PREFIX/test-warehouse/alltypes_rc/year=2009/month=5'
 ---- RESULTS
+'New location has been set for the specified partition.'
 ====
 ---- QUERY
 alter table alltypes_test partition(month=cast(2+3 as int), year=2009)
@@ -661,6 +684,7 @@ STRING, STRING, BIGINT, BIGINT, STRING, STRING, STRING, 
STRING, STRING, STRING
 # IMPALA-1016: Testing scanning newly added columns
 CREATE TABLE imp1016 (string1 string)
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 INSERT INTO imp1016 VALUES ('test')
@@ -670,6 +694,7 @@ INSERT INTO imp1016 VALUES ('test')
 ---- QUERY
 ALTER TABLE imp1016 ADD COLUMNS (string2 string)
 ---- RESULTS
+'New column(s) have been added to the table.'
 ====
 ---- QUERY
 DESCRIBE imp1016
@@ -718,6 +743,7 @@ bigint
 # Create a larger table to test scanning newly added columns
 CREATE TABLE imp1016Large (string1 string)
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # There is a codepath that operates on chunks of 1024 tuples, inserting
@@ -729,6 +755,7 @@ INSERT INTO imp1016Large SELECT 'test' FROM 
functional.alltypes LIMIT 2000
 ---- QUERY
 ALTER TABLE imp1016Large ADD COLUMNS (string2 string)
 ---- RESULTS
+'New column(s) have been added to the table.'
 ====
 ---- QUERY
 DESCRIBE imp1016Large
@@ -801,6 +828,7 @@ STRING, STRING, BIGINT, BIGINT, BIGINT, DOUBLE
 ---- QUERY
 drop table $DATABASE2.mv2
 ---- RESULTS
+'Table has been dropped.'
 ====
 ---- QUERY
 show tables in $DATABASE2 like '*mv*'
@@ -855,6 +883,7 @@ STRING, STRING, BIGINT, BIGINT, BIGINT, DOUBLE
 ---- QUERY
 drop table $DATABASE2.mv2
 ---- RESULTS
+'Table has been dropped.'
 ====
 ---- QUERY
 show tables in $DATABASE2 like '*mv*'

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/chars-tmp-tables.test
----------------------------------------------------------------------
diff --git 
a/testdata/workloads/functional-query/queries/QueryTest/chars-tmp-tables.test 
b/testdata/workloads/functional-query/queries/QueryTest/chars-tmp-tables.test
index f6dc4c4..6840951 100644
--- 
a/testdata/workloads/functional-query/queries/QueryTest/chars-tmp-tables.test
+++ 
b/testdata/workloads/functional-query/queries/QueryTest/chars-tmp-tables.test
@@ -2,6 +2,7 @@
 ---- QUERY
 create table test_char_tmp (c char(5))
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 insert into test_char_tmp select cast("hello" as char(5))
@@ -64,6 +65,7 @@ string
 ---- QUERY
 create table test_varchar_tmp (vc varchar(5))
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 insert into test_varchar_tmp values (cast("hello" as varchar(5)))
@@ -101,6 +103,7 @@ string
 create table allchars
 (cshort char(5), clong char(140), vc varchar(5))
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 insert into allchars values (cast("123456" as char(5)), cast("123456" as 
char(140)),
@@ -117,6 +120,7 @@ char,char,string
 create table allchars_par
 (cshort char(5), clong char(140), vc varchar(5)) stored as parquet
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 insert into allchars_par values (cast("123456" as char(5)), cast("123456" as 
char(140)),
@@ -193,6 +197,7 @@ test_char_nulls ( c20 char(20),
                   c120 char(120),
                   c140 char(140))
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 insert into test_char_nulls

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/compute-stats-incremental.test
----------------------------------------------------------------------
diff --git 
a/testdata/workloads/functional-query/queries/QueryTest/compute-stats-incremental.test
 
b/testdata/workloads/functional-query/queries/QueryTest/compute-stats-incremental.test
index ddc6da3..50eb24d 100644
--- 
a/testdata/workloads/functional-query/queries/QueryTest/compute-stats-incremental.test
+++ 
b/testdata/workloads/functional-query/queries/QueryTest/compute-stats-incremental.test
@@ -609,6 +609,7 @@ STRING
 # The table was not changed. Validate that the next compute incremental stats 
is a no-op.
 compute incremental stats complextypestbl_part;
 ---- RESULTS
+'No partitions selected for incremental stats update.'
 ---- ERRORS
 No partitions selected for incremental stats update
 ====

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/compute-stats.test
----------------------------------------------------------------------
diff --git 
a/testdata/workloads/functional-query/queries/QueryTest/compute-stats.test 
b/testdata/workloads/functional-query/queries/QueryTest/compute-stats.test
index b7494f0..ce147e4 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/compute-stats.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/compute-stats.test
@@ -70,6 +70,7 @@ STRING, STRING, BIGINT, BIGINT, BIGINT, DOUBLE
 # Adding a column shouldn't cause the stats to be dropped.
 alter table alltypes add columns (new_col int)
 ---- RESULTS
+'New column(s) have been added to the table.'
 ====
 ---- QUERY
 show column stats alltypes
@@ -99,6 +100,7 @@ STRING, STRING, BIGINT, BIGINT, BIGINT, DOUBLE
 # HIVE-15075 is resolved.
 alter table alltypes change new_col new_col2 int
 ---- RESULTS
+'Column has been altered.'
 ====
 ---- QUERY
 show column stats alltypes
@@ -126,6 +128,7 @@ STRING, STRING, BIGINT, BIGINT, BIGINT, DOUBLE
 # Removing a column shouldn't cause the stats to be dropped.
 alter table alltypes drop column new_col2
 ---- RESULTS
+'Column has been dropped.'
 ====
 ---- QUERY
 show column stats alltypes
@@ -345,6 +348,7 @@ STRING, STRING, BIGINT, BIGINT, BIGINT, DOUBLE
 # Add partitions with NULL values and check for stats.
 alter table alltypes add partition (year=NULL, month=NULL)
 ---- RESULTS
+'New partition has been added to the table.'
 ====
 ---- QUERY
 show column stats alltypes
@@ -370,6 +374,7 @@ STRING, STRING, BIGINT, BIGINT, BIGINT, DOUBLE
 ---- QUERY
 alter table alltypes add partition (year=2011, month=NULL)
 ---- RESULTS
+'New partition has been added to the table.'
 ====
 ---- QUERY
 show column stats alltypes
@@ -1025,9 +1030,13 @@ STRING, BIGINT, BIGINT, STRING, STRING, STRING, STRING, 
STRING, STRING
 insert into table empty_partitioned partition (j=2) select 1;
 ====
 ---- QUERY
+drop stats empty_partitioned
+---- RESULTS
+'Stats have been dropped.'
+====
+---- QUERY
 # Verify partition stats work with empty and non-empty partition.
-drop stats empty_partitioned;
-compute stats empty_partitioned;
+compute stats empty_partitioned
 ---- RESULTS
 'Updated 2 partition(s) and 1 column(s).'
 ---- TYPES

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/create-database.test
----------------------------------------------------------------------
diff --git 
a/testdata/workloads/functional-query/queries/QueryTest/create-database.test 
b/testdata/workloads/functional-query/queries/QueryTest/create-database.test
index 1fcf85c..765bca8 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/create-database.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/create-database.test
@@ -2,6 +2,7 @@
 ---- QUERY
 create database $DATABASE_2 comment "For testing"
 ---- RESULTS
+'Database has been created.'
 ====
 ---- QUERY
 show databases like "$DATABASE_2"
@@ -15,11 +16,13 @@ STRING, STRING
 # IF NOT EXISTS is specified.
 create database if not exists $DATABASE_2
 ---- RESULTS
+'Database already exists.'
 ====
 ---- QUERY
 # Test dropping the database.
 drop database $DATABASE_2
 ---- RESULTS
+'Database has been dropped.'
 ====
 ---- QUERY
 show databases like "$DATABASE_2"
@@ -31,6 +34,7 @@ STRING, STRING
 # Dropping a non-existent databases is ok with IF EXISTS
 drop database if exists $DATABASE_2
 ---- RESULTS
+'Database has been dropped.'
 ====
 ---- QUERY
 # Test DROP DATABASE ... CASCADE
@@ -50,6 +54,7 @@ create function if not exists $DATABASE_cascade.f1() returns 
string
 create aggregate function if not exists $DATABASE_cascade.f2(int, string) 
RETURNS int
   location '$FILESYSTEM_PREFIX/test-warehouse/libTestUdas.so' 
UPDATE_FN='TwoArgUpdate'
 ---- RESULTS
+'Function has been created.'
 ====
 ---- QUERY
 show tables in $DATABASE_cascade
@@ -80,6 +85,7 @@ STRING, STRING, STRING, STRING
 # as the database itself.
 drop database $DATABASE_cascade cascade
 ---- RESULTS
+'Database has been dropped.'
 ====
 ---- QUERY
 show databases like '$DATABASE_cascade'
@@ -99,6 +105,7 @@ STRING,STRING
 ---- QUERY
 drop database $DATABASE_restrict restrict
 ---- RESULTS
+'Database has been dropped.'
 ====
 ---- QUERY
 show databases like '$DATABASE_restrict'

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/create-table-like-file.test
----------------------------------------------------------------------
diff --git 
a/testdata/workloads/functional-query/queries/QueryTest/create-table-like-file.test
 
b/testdata/workloads/functional-query/queries/QueryTest/create-table-like-file.test
index bd54f3d..7a80602 100644
--- 
a/testdata/workloads/functional-query/queries/QueryTest/create-table-like-file.test
+++ 
b/testdata/workloads/functional-query/queries/QueryTest/create-table-like-file.test
@@ -3,6 +3,7 @@
 create table $DATABASE.temp_decimal_table like parquet
 '$FILESYSTEM_PREFIX/test-warehouse/schemas/decimal.parquet'
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 describe $DATABASE.temp_decimal_table
@@ -17,6 +18,7 @@ STRING, STRING, STRING
 create table $DATABASE.like_zipcodes_file like parquet
 '$FILESYSTEM_PREFIX/test-warehouse/schemas/zipcode_incomes.parquet'
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 describe $DATABASE.like_zipcodes_file
@@ -33,6 +35,7 @@ STRING, STRING, STRING
 create table $DATABASE.like_alltypestiny_file like parquet
 '$FILESYSTEM_PREFIX/test-warehouse/schemas/alltypestiny.parquet'
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 describe $DATABASE.like_alltypestiny_file
@@ -59,6 +62,7 @@ create external table $DATABASE.like_enumtype_file like 
parquet
 STORED AS PARQUET
 LOCATION '$FILESYSTEM_PREFIX/test-warehouse/schemas/enum'
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 describe $DATABASE.like_enumtype_file
@@ -81,6 +85,7 @@ STRING, STRING
 create table if not exists $DATABASE.like_alltypestiny_file like parquet
 '$FILESYSTEM_PREFIX/test-warehouse/schemas/zipcode_incomes.parquet'
 ---- RESULTS
+'Table already exists.'
 ====
 ---- QUERY
 # Should not have changed since last statement was IF NOT EXISTS.
@@ -103,11 +108,13 @@ STRING, STRING, STRING
 ---- QUERY
 drop table if exists allcomplextypes_clone
 ---- RESULTS
+'Table does not exist.'
 ====
 ---- QUERY
 create table allcomplextypes_clone like functional.allcomplextypes
 stored as parquet
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 describe allcomplextypes_clone
@@ -133,15 +140,18 @@ STRING, STRING, STRING
 ---- QUERY
 drop table allcomplextypes_clone
 ---- RESULTS
+'Table has been dropped.'
 ====
 ---- QUERY
 drop table if exists $DATABASE.temp_legacy_table
 ---- RESULTS
+'Table does not exist.'
 ====
 ---- QUERY
 create table $DATABASE.temp_legacy_table like parquet
 '$FILESYSTEM_PREFIX/test-warehouse/schemas/legacy_nested.parquet'
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 describe $DATABASE.temp_legacy_table
@@ -162,15 +172,18 @@ STRING, STRING, STRING
 ---- QUERY
 drop table if exists $DATABASE.temp_legacy_table
 ---- RESULTS
+'Table has been dropped.'
 ====
 ---- QUERY
 drop table if exists $DATABASE.temp_modern_table
 ---- RESULTS
+'Table does not exist.'
 ====
 ---- QUERY
 create table $DATABASE.temp_modern_table like parquet
 '$FILESYSTEM_PREFIX/test-warehouse/schemas/modern_nested.parquet'
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 describe $DATABASE.temp_modern_table
@@ -191,6 +204,7 @@ STRING, STRING, STRING
 ---- QUERY
 drop table if exists $DATABASE.temp_modern_table
 ---- RESULTS
+'Table has been dropped.'
 ====
 ---- QUERY
 # Test adding sort.columns when creating a table like a parquet file.

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/create-table-like-table.test
----------------------------------------------------------------------
diff --git 
a/testdata/workloads/functional-query/queries/QueryTest/create-table-like-table.test
 
b/testdata/workloads/functional-query/queries/QueryTest/create-table-like-table.test
index 0a4df27..ee16c37 100644
--- 
a/testdata/workloads/functional-query/queries/QueryTest/create-table-like-table.test
+++ 
b/testdata/workloads/functional-query/queries/QueryTest/create-table-like-table.test
@@ -4,6 +4,7 @@
 create table alltypes_test like functional_seq_snap.alltypes
 stored as parquet
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # Make sure no data exists for this table
@@ -35,6 +36,7 @@ BIGINT
 # CREATE TABLE LIKE on a view
 create table like_view like functional.view_view
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 describe like_view
@@ -67,6 +69,7 @@ BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING
 ---- QUERY
 create table like_view_parquet like functional.view_view stored as parquet
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 show table stats like_view_parquet
@@ -82,6 +85,7 @@ BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING
 create external table jointbl_rc_like like functional_rc_gzip.jointbl
 location '$FILESYSTEM_PREFIX/test-warehouse/jointbl_rc_gzip'
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # should get some results back
@@ -97,6 +101,7 @@ BIGINT, STRING, INT, INT
 # CREATE TABLE LIKE on unpartitioned table.
 create table jointbl_like like functional.jointbl
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # Make sure the new table can be queried and no data exists for this table.
@@ -110,11 +115,13 @@ BIGINT
 # No error is thrown when IF NOT EXISTS is specified and the table already 
exists.
 create table if not exists jointbl_like like functional.jointbl
 ---- RESULTS
+'Table already exists.'
 ====
 ---- QUERY
 # IF NOT EXISTS also applies when the src table is the same as the new table.
 create table if not exists jointbl_like like jointbl_like
 ---- RESULTS
+'Table already exists.'
 ====
 ---- QUERY
 insert overwrite table jointbl_like
@@ -155,6 +162,7 @@ create table no_avro_schema (
 partitioned by (year int, month int)
 stored as avro
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 describe no_avro_schema
@@ -182,6 +190,7 @@ STRING, STRING, STRING
 # Test creating an Avro table without an Avro schema via CREATE TABLE LIKE 
(IMPALA-1813)
 create table like_no_avro_schema like no_avro_schema stored as avro
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 describe like_no_avro_schema
@@ -208,10 +217,12 @@ STRING, STRING, STRING
 ---- QUERY
 drop table like_no_avro_schema
 ---- RESULTS
+'Table has been dropped.'
 ====
 ---- QUERY
 drop table no_avro_schema
 ---- RESULTS
+'Table has been dropped.'
 ====
 ---- QUERY
 # Test setting sort.columns when using create table like.

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/create-table.test
----------------------------------------------------------------------
diff --git 
a/testdata/workloads/functional-query/queries/QueryTest/create-table.test 
b/testdata/workloads/functional-query/queries/QueryTest/create-table.test
index 2661394..7f91adc 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/create-table.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/create-table.test
@@ -2,6 +2,7 @@
 ---- QUERY
 create table $DATABASE.testtbl(i int, s string COMMENT 'String col') STORED AS 
TEXTFILE
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # Make sure creating a table with the same name doesn't throw an error when
@@ -13,6 +14,7 @@ ESCAPED BY '\\'
 LINES TERMINATED BY '\n'
 STORED AS TEXTFILE
 ---- RESULTS
+'Table already exists.'
 ====
 ---- QUERY
 show tables in $DATABASE
@@ -54,6 +56,7 @@ INT, STRING
 ---- QUERY
 create table $DATABASE.testtbl_part(i int, s string) PARTITIONED BY (id int 
comment 'C')
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # Partition columns are displayed as part of DESCRIBE <table>
@@ -105,6 +108,7 @@ STRING
 # Make sure we create the table in the proper database after a "use"
 create table testtbl2(f float, d double) ROW FORMAT DELIMITED FIELDS 
TERMINATED BY '|'
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 show tables
@@ -118,6 +122,7 @@ STRING
 ---- QUERY
 drop table testtbl2
 ---- RESULTS
+'Table has been dropped.'
 ====
 ---- QUERY
 show tables
@@ -134,6 +139,7 @@ with serdeproperties
 
('avro.schema.url'='$FILESYSTEM_PREFIX/test-warehouse/avro_schemas/functional/alltypes.json')
 stored as avro
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 describe avro_alltypes_nopart
@@ -160,6 +166,7 @@ with serdeproperties
 
('avro.schema.url'='$FILESYSTEM_PREFIX/test-warehouse/avro_schemas/functional/alltypes.json')
 stored as avro
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 describe avro_alltypes_part
@@ -183,18 +190,22 @@ STRING, STRING, STRING
 ---- QUERY
 drop table avro_alltypes_part
 ---- RESULTS
+'Table has been dropped.'
 ====
 ---- QUERY
 drop table avro_alltypes_nopart
 ---- RESULTS
+'Table has been dropped.'
 ====
 ---- QUERY
 drop table testtbl
 ---- RESULTS
+'Table has been dropped.'
 ====
 ---- QUERY
 drop table testtbl_part
 ---- RESULTS
+'Table has been dropped.'
 ====
 ---- QUERY
 show tables
@@ -205,6 +216,7 @@ STRING
 ---- QUERY
 drop table if exists non_existent_db.tbl
 ---- RESULTS
+'Database does not exist: non_existent_db'
 ====
 ---- QUERY
 # Test table creation with tblproperty and serdeproperty lengths just within 
limits
@@ -219,17 +231,20 @@ with serdeproperties(
 
'valuevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluev
 
aluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevalueval
 
uevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevalue
 
valuevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevaluevalueva
 luevaluevalue'
 )
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # IMPALA-1740: Test setting the skip.header.line.count tblproperty
 create table skip_header_test_a (i1 integer) 
tblproperties('skip.header.line.count'='2')
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # IMPALA-1740: Test setting the skip.header.line.count tblproperty on a 
Parquet table
 create table skip_header_test_d (i1 integer) stored as parquet
 tblproperties('skip.header.line.count'='2')
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # IMPALA-1740: Test setting an invalid skip.header.line.count tblproperty

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/delimited-latin-text.test
----------------------------------------------------------------------
diff --git 
a/testdata/workloads/functional-query/queries/QueryTest/delimited-latin-text.test
 
b/testdata/workloads/functional-query/queries/QueryTest/delimited-latin-text.test
index 460aaac..004feff 100644
--- 
a/testdata/workloads/functional-query/queries/QueryTest/delimited-latin-text.test
+++ 
b/testdata/workloads/functional-query/queries/QueryTest/delimited-latin-text.test
@@ -18,6 +18,7 @@ STRING,STRING,INT,INT
 # create new tables like the ones above to test inserting
 create table tecn like functional.text_thorn_ecirc_newline;
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # insert data into tecn table and check results
@@ -35,4 +36,4 @@ select * from tecn
 'efg','xyz',3,4
 ---- TYPES
 STRING,STRING,INT,INT
-====
\ No newline at end of file
+====

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/delimited-text.test
----------------------------------------------------------------------
diff --git 
a/testdata/workloads/functional-query/queries/QueryTest/delimited-text.test 
b/testdata/workloads/functional-query/queries/QueryTest/delimited-text.test
index 188d54d..9dde519 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/delimited-text.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/delimited-text.test
@@ -34,6 +34,7 @@ STRING,STRING,INT,INT
 create table cbn like functional.text_comma_backslash_newline;
 create table dhp like functional.text_dollar_hash_pipe;
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # insert data into cbn table and check results
@@ -70,4 +71,4 @@ select * from dhp
 'abc #$#$ abc','xyz $#$# xyz',5,6
 ---- TYPES
 STRING,STRING,INT,INT
-====
\ No newline at end of file
+====

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/describe-path.test
----------------------------------------------------------------------
diff --git 
a/testdata/workloads/functional-query/queries/QueryTest/describe-path.test 
b/testdata/workloads/functional-query/queries/QueryTest/describe-path.test
index 2547267..d6f041c 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/describe-path.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/describe-path.test
@@ -90,6 +90,7 @@ create table if not exists nested_structs (
   map_array_map_struct_col
   map<string, array<map<string, struct<f1:string, f2:int>>>>)
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 describe nested_structs

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/functions-ddl.test
----------------------------------------------------------------------
diff --git 
a/testdata/workloads/functional-query/queries/QueryTest/functions-ddl.test 
b/testdata/workloads/functional-query/queries/QueryTest/functions-ddl.test
index d41d9da..a097c61 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/functions-ddl.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/functions-ddl.test
@@ -3,12 +3,21 @@
 # Add functions and test function overloading and scoping.
 create function $DATABASE.fn() RETURNS int
 LOCATION '$FILESYSTEM_PREFIX/test-warehouse/libTestUdfs.so' SYMBOL='Fn'
+---- RESULTS
+'Function has been created.'
 ====
 ---- QUERY
 create function $DATABASE.fn(int) RETURNS double
 LOCATION '$FILESYSTEM_PREFIX/test-warehouse/libTestUdfs.so' SYMBOL='Fn'
 ====
 ---- QUERY
+# Test IF NOT EXISTS
+create function if not exists $DATABASE.fn(int) RETURNS double
+LOCATION '$FILESYSTEM_PREFIX/test-warehouse/libTestUdfs.so' SYMBOL='Fn'
+---- RESULTS
+'Function already exists.'
+====
+---- QUERY
 create function $DATABASE.fn(int, string) RETURNS int
 LOCATION '$FILESYSTEM_PREFIX/test-warehouse/libTestUdfs.so' SYMBOL='Fn'
 ====
@@ -149,6 +158,8 @@ STRING
 ====
 ---- QUERY
 drop function $DATABASE.fn2(int, string)
+---- RESULTS
+'Function has been dropped.'
 ====
 ---- QUERY
 show functions
@@ -199,6 +210,11 @@ STRING, STRING, STRING, STRING
 drop function fn()
 ====
 ---- QUERY
+drop function if exists fn()
+---- RESULTS
+'Function does not exist.'
+====
+---- QUERY
 show functions;
 ---- LABELS
 return type, signature, binary type, is persistent

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/grant_revoke.test
----------------------------------------------------------------------
diff --git 
a/testdata/workloads/functional-query/queries/QueryTest/grant_revoke.test 
b/testdata/workloads/functional-query/queries/QueryTest/grant_revoke.test
index a69a93f..e7f8f2d 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/grant_revoke.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/grant_revoke.test
@@ -2,18 +2,22 @@
 ---- QUERY
 create role grant_revoke_test_ALL_SERVER
 ---- RESULTS
+'Role has been created.'
 ====
 ---- QUERY
 create role grant_revoke_test_ALL_TEST_DB
 ---- RESULTS
+'Role has been created.'
 ====
 ---- QUERY
 create role grant_revoke_test_SELECT_INSERT_TEST_TBL
 ---- RESULTS
+'Role has been created.'
 ====
 ---- QUERY
 create role grant_revoke_test_ALL_URI
 ---- RESULTS
+'Role has been created.'
 ====
 ---- QUERY
 # Shows all roles in the system
@@ -489,10 +493,12 @@ root
 REVOKE ROLE grant_revoke_test_ALL_URI FROM GROUP `$GROUP_NAME`;
 REVOKE ROLE grant_revoke_test_SELECT_INSERT_TEST_TBL FROM GROUP `$GROUP_NAME`;
 ---- RESULTS
+'Role has been revoked.'
 ====
 ---- QUERY
 GRANT ROLE grant_revoke_test_ALL_SERVER TO GROUP `$GROUP_NAME`
 ---- RESULTS
+'Role has been granted.'
 ====
 ---- QUERY
 show current roles
@@ -505,10 +511,12 @@ STRING
 # Create a table with multiple columns to test column-level security.
 create table grant_rev_db.test_tbl3(a int, b int, c int, d int, e int) 
partitioned by (x int, y int)
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 GRANT SELECT (a, b, x) ON TABLE grant_rev_db.test_tbl3 TO 
grant_revoke_test_ALL_SERVER
 ---- RESULTS
+'Privilege(s) have been granted.'
 ====
 ---- QUERY
 show grant role grant_revoke_test_ALL_SERVER
@@ -525,6 +533,7 @@ STRING, STRING, STRING, STRING, STRING, STRING, BOOLEAN, 
STRING
 ---- QUERY
 GRANT SELECT (c, d, y) ON TABLE grant_rev_db.test_tbl3 TO 
grant_revoke_test_ALL_SERVER
 ---- RESULTS
+'Privilege(s) have been granted.'
 ====
 ---- QUERY
 show grant role grant_revoke_test_ALL_SERVER
@@ -544,6 +553,7 @@ STRING, STRING, STRING, STRING, STRING, STRING, BOOLEAN, 
STRING
 ---- QUERY
 GRANT SELECT (a, a, e, x) ON TABLE grant_rev_db.test_tbl3 TO 
grant_revoke_test_ALL_SERVER
 ---- RESULTS
+'Privilege(s) have been granted.'
 ====
 ---- QUERY
 show grant role grant_revoke_test_ALL_SERVER
@@ -565,6 +575,7 @@ STRING, STRING, STRING, STRING, STRING, STRING, BOOLEAN, 
STRING
 # Revoke SELECT privileges from columns
 REVOKE SELECT (a, b, b, y) ON TABLE grant_rev_db.test_tbl3 FROM 
grant_revoke_test_ALL_SERVER
 ---- RESULTS
+'Privilege(s) have been revoked.'
 ====
 ---- QUERY
 show grant role grant_revoke_test_ALL_SERVER
@@ -582,6 +593,7 @@ STRING, STRING, STRING, STRING, STRING, STRING, BOOLEAN, 
STRING
 ---- QUERY
 REVOKE SELECT (a, b, c, x) ON TABLE grant_rev_db.test_tbl3 FROM 
grant_revoke_test_ALL_SERVER
 ---- RESULTS
+'Privilege(s) have been revoked.'
 ====
 ---- QUERY
 show grant role grant_revoke_test_ALL_SERVER
@@ -597,6 +609,7 @@ STRING, STRING, STRING, STRING, STRING, STRING, BOOLEAN, 
STRING
 ---- QUERY
 REVOKE SELECT (a, b, c, d, e) ON TABLE grant_rev_db.test_tbl3 FROM 
grant_revoke_test_ALL_SERVER;
 ---- RESULTS
+'Privilege(s) have been revoked.'
 ====
 ---- QUERY
 show grant role grant_revoke_test_ALL_SERVER
@@ -613,6 +626,7 @@ GRANT ROLE grant_revoke_test_ROOT TO GROUP root;
 GRANT SELECT ON TABLE grant_rev_db.test_tbl3 TO grant_revoke_test_ROOT;
 REVOKE ALL ON DATABASE functional FROM grant_revoke_test_ROOT;
 ---- RESULTS
+'Privilege(s) have been revoked.'
 ====
 ---- USER
 root
@@ -635,17 +649,20 @@ User 'root' does not have privileges to execute: 
GRANT_PRIVILEGE
 ---- QUERY
 REVOKE SELECT ON TABLE grant_rev_db.test_tbl3 FROM grant_revoke_test_ROOT
 ---- RESULTS
+'Privilege(s) have been revoked.'
 ====
 ---- QUERY
 # Grant SELECT on table to 'root' with 'WITH GRANT' option.
 GRANT SELECT ON TABLE grant_rev_db.test_tbl3 TO grant_revoke_test_ROOT WITH 
GRANT OPTION
 ---- RESULTS
+'Privilege(s) have been granted.'
 ====
 ---- USER
 root
 ---- QUERY
 GRANT SELECT (a) ON TABLE grant_rev_db.test_tbl3 TO grant_revoke_test_ROOT
 ---- RESULTS
+'Privilege(s) have been granted.'
 ====
 ---- USER
 root
@@ -662,6 +679,7 @@ STRING, STRING, STRING, STRING, STRING, STRING, BOOLEAN, 
STRING
 ---- QUERY
 GRANT SELECT (a, c, e) ON TABLE grant_rev_db.test_tbl3 TO 
grant_revoke_test_ALL_SERVER  WITH GRANT OPTION
 ---- RESULTS
+'Privilege(s) have been granted.'
 ====
 ---- QUERY
 show grant role grant_revoke_test_ALL_SERVER
@@ -678,6 +696,7 @@ STRING, STRING, STRING, STRING, STRING, STRING, BOOLEAN, 
STRING
 ---- QUERY
 REVOKE GRANT OPTION FOR SELECT (a, c) ON TABLE grant_rev_db.test_tbl3 FROM 
grant_revoke_test_ALL_SERVER
 ---- RESULTS
+'Privilege(s) have been revoked.'
 ====
 ---- QUERY
 # TODO: Add a test case that exercises the cascading effect of REVOKE ALL.
@@ -699,6 +718,7 @@ revoke role grant_revoke_test_ALL_SERVER from group 
`$GROUP_NAME`
 # Test 'grant all on server' with explicit server name specified.
 create role grant_revoke_test_ALL_SERVER1
 ---- RESULTS
+'Role has been created.'
 ====
 ---- QUERY
 grant all on server server1 to grant_revoke_test_ALL_SERVER1
@@ -745,6 +765,7 @@ STRING, STRING, STRING, STRING, STRING, STRING, BOOLEAN, 
STRING
 # to a table in the database
 grant role grant_revoke_test_ALL_SERVER to group `$GROUP_NAME`
 ---- RESULTS
+'Role has been granted.'
 ====
 ---- QUERY
 create role grant_revoke_test_COLUMN_PRIV
@@ -800,12 +821,15 @@ STRING,STRING
 ---- QUERY
 grant role grant_revoke_test_ALL_SERVER to group `$GROUP_NAME`
 ---- RESULTS
+'Role has been granted.'
 ====
 ---- QUERY
 drop database if exists grant_rev_db cascade
 ====
 ---- QUERY
 revoke role grant_revoke_test_ALL_SERVER from group `$GROUP_NAME`
+---- RESULTS
+'Role has been revoked.'
 ====
 ---- QUERY
 revoke role grant_revoke_test_COLUMN_PRIV from group `$GROUP_NAME`
@@ -818,4 +842,5 @@ drop role grant_revoke_test_ALL_URI;
 drop role grant_revoke_test_ROOT;
 drop role grant_revoke_test_COLUMN_PRIV;
 ---- RESULTS
+'Role has been dropped.'
 ====

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/grant_revoke_kudu.test
----------------------------------------------------------------------
diff --git 
a/testdata/workloads/functional-query/queries/QueryTest/grant_revoke_kudu.test 
b/testdata/workloads/functional-query/queries/QueryTest/grant_revoke_kudu.test
index a3b9354..f51075c 100644
--- 
a/testdata/workloads/functional-query/queries/QueryTest/grant_revoke_kudu.test
+++ 
b/testdata/workloads/functional-query/queries/QueryTest/grant_revoke_kudu.test
@@ -2,10 +2,12 @@
 ---- QUERY
 create role grant_revoke_test_ALL_SERVER
 ---- RESULTS
+'Role has been created.'
 ====
 ---- QUERY
 create role grant_revoke_test_ALL_TEST_DB
 ---- RESULTS
+'Role has been created.'
 ====
 ---- QUERY
 show roles
@@ -156,6 +158,7 @@ does not have privileges to access: grant_rev_db.kudu_tbl
 ---- QUERY
 grant select(a) on table grant_rev_db.kudu_tbl to grant_revoke_test_KUDU
 ---- RESULTS
+'Privilege(s) have been granted.'
 ====
 ---- QUERY
 grant ALL on table grant_rev_db.kudu_tbl to grant_revoke_test_KUDU
@@ -184,4 +187,5 @@ drop role grant_revoke_test_ALL_SERVER;
 drop role grant_revoke_test_ALL_TEST_DB;
 drop role grant_revoke_test_KUDU;
 ---- RESULTS
+'Role has been dropped.'
 ====

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats-incremental.test
----------------------------------------------------------------------
diff --git 
a/testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats-incremental.test
 
b/testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats-incremental.test
index a217de7..a0ecb18 100644
--- 
a/testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats-incremental.test
+++ 
b/testdata/workloads/functional-query/queries/QueryTest/hbase-compute-stats-incremental.test
@@ -2,6 +2,7 @@
 ---- QUERY
 create table alltypessmall_hbase like functional_hbase.alltypessmall
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 compute incremental stats alltypessmall_hbase

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/insert_bad_expr.test
----------------------------------------------------------------------
diff --git 
a/testdata/workloads/functional-query/queries/QueryTest/insert_bad_expr.test 
b/testdata/workloads/functional-query/queries/QueryTest/insert_bad_expr.test
index 9d6b9d1..43edb36 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/insert_bad_expr.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/insert_bad_expr.test
@@ -6,6 +6,7 @@ create function if not exists twenty_one_args(int, int, int, 
int, int, int,
 location '$FILESYSTEM_PREFIX/test-warehouse/libTestUdfs.so'
 symbol='TwentyOneArgs';
 ---- RESULTS
+'Function has been created.'
 ====
 ---- QUERY
 # Regression test for IMPALA-6262: failure to initialize the output expressions
@@ -21,5 +22,6 @@ Cannot interpret native UDF 'twenty_one_args': number of 
arguments is more than
 drop function twenty_one_args(int, int, int, int, int, int, int, int,
     int, int, int, int, int, int, int, int, int, int, int, int, int);
 ---- RESULTS
+'Function has been dropped.'
 ====
 

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/insert_permutation.test
----------------------------------------------------------------------
diff --git 
a/testdata/workloads/functional-query/queries/QueryTest/insert_permutation.test 
b/testdata/workloads/functional-query/queries/QueryTest/insert_permutation.test
index dfdb2ed..696644f 100644
--- 
a/testdata/workloads/functional-query/queries/QueryTest/insert_permutation.test
+++ 
b/testdata/workloads/functional-query/queries/QueryTest/insert_permutation.test
@@ -3,6 +3,7 @@
 create database insert_permutation_test location
 '$FILESYSTEM_PREFIX/test-warehouse/insert_permutation_test'
 ---- RESULTS
+'Database has been created.'
 ====
 ---- QUERY
 use insert_permutation_test
@@ -14,6 +15,7 @@ create table perm_part(int_col1 int, string_col string) 
partitioned by (p1 int,
 create table parquet_part(int_col1 int, string_col string)
 partitioned by (p1 int, p2 string) stored as parquet;
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # Simple non-permutation

http://git-wip-us.apache.org/repos/asf/impala/blob/2ee914d5/testdata/workloads/functional-query/queries/QueryTest/kudu_alter.test
----------------------------------------------------------------------
diff --git 
a/testdata/workloads/functional-query/queries/QueryTest/kudu_alter.test 
b/testdata/workloads/functional-query/queries/QueryTest/kudu_alter.test
index 305ccf1..85937fa 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/kudu_alter.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/kudu_alter.test
@@ -3,6 +3,7 @@
 create table simple (id int primary key, name string, valf float, vali bigint)
   partition by hash (id) partitions 3 stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # Hash partitions cannot be enumerated as range partitions
@@ -44,6 +45,7 @@ ImpalaRuntimeException: Kudu table 'impala::$DATABASE.simple' 
does not exist on
 ---- QUERY
 alter table simple rename to simple_new;
 ---- RESULTS
+'Renaming was successful.'
 ====
 ---- QUERY
 select count(*) from simple_new;
@@ -57,6 +59,7 @@ BIGINT
 create table tbl_to_alter (id int primary key, name string null, vali bigint 
not null)
   partition by range (id) (partition 1 < values <= 10) stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # Verify partition layout
@@ -68,6 +71,7 @@ show range partitions tbl_to_alter;
 # Add a range partition
 alter table tbl_to_alter add range partition 10 < values <= 20
 ---- RESULTS
+'Range partition has been added.'
 ====
 ---- QUERY
 # Verify partition layout
@@ -93,6 +97,7 @@ INT,STRING,BIGINT
 # Add a singleton range partition
 alter table tbl_to_alter add range partition value = 100
 ---- RESULTS
+'Range partition has been added.'
 ====
 ---- QUERY
 # Verify partition layout
@@ -120,6 +125,7 @@ INT,STRING,BIGINT
 # Add an unbounded range partition
 alter table tbl_to_alter add range partition 1000 < values
 ---- RESULTS
+'Range partition has been added.'
 ====
 ---- QUERY
 # Verify partition layout
@@ -141,11 +147,13 @@ NonRecoverableException: New range partition conflicts 
with existing range parti
 # to hide the error
 alter table tbl_to_alter add if not exists range partition 10 < values <= 30
 ---- RESULTS
+'Range partition has been added.'
 ====
 ---- QUERY
 # Drop one of the recently inserted partitions
 alter table tbl_to_alter drop range partition value = 100
 ---- RESULTS
+'Range partition has been dropped.'
 ====
 ---- QUERY
 # Verify partition layout
@@ -167,6 +175,7 @@ INT,STRING,BIGINT
 # Drop an existing range partition
 alter table tbl_to_alter drop range partition 11 <= values < 21
 ---- RESULTS
+'Range partition has been dropped.'
 ====
 ---- QUERY
 # Verify partition layout
@@ -180,6 +189,7 @@ show range partitions tbl_to_alter;
 alter table tbl_to_alter drop range partition 1 < values <= 10;
 alter table tbl_to_alter drop range partition 1000 < values
 ---- RESULTS
+'Range partition has been dropped.'
 ====
 ---- QUERY
 # Verify partition layout
@@ -209,6 +219,7 @@ alter table tbl_to_alter add range partition 1 < values <= 
20;
 alter table tbl_to_alter add columns (new_col1 int not null default 10,
   new_col2 bigint not null default 1000)
 ---- RESULTS
+'Column has been added/replaced.'
 ====
 ---- QUERY
 # Verify partition layout
@@ -261,6 +272,7 @@ INT,STRING,BIGINT,INT,BIGINT
 # Add nullable columns: with and without a default
 alter table tbl_to_alter add columns (new_col3 string null, new_col4 int null 
default -1)
 ---- RESULTS
+'Column has been added/replaced.'
 ====
 ---- QUERY
 # Add a row
@@ -307,6 +319,7 @@ A new non-null column must have a default value
 # Drop a column
 alter table tbl_to_alter drop column vali
 ---- RESULTS
+'Column has been dropped.'
 ====
 ---- QUERY
 # Retrieve table rows after column got dropped
@@ -330,6 +343,7 @@ NonRecoverableException: cannot remove a key column
 # Rename a column
 alter table tbl_to_alter change column new_col3 last_name string
 ---- RESULTS
+'Column has been altered.'
 ====
 ---- QUERY
 # Ensure the renamed column is accessible
@@ -355,6 +369,7 @@ BIGINT
 # Rename the Impala table
 alter table tbl_to_alter rename to kudu_tbl_to_alter
 ---- RESULTS
+'Renaming was successful.'
 ====
 ---- QUERY
 # Ensure the Impala table is accessible after it got renamed
@@ -421,6 +436,7 @@ partition by range (
             cast('2009-01-02 00:00:00' as timestamp)
 ) stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 show range partitions ts_ranges
@@ -434,6 +450,7 @@ alter table ts_ranges add range partition
 cast('2009-01-02 00:00:00' as timestamp) <= VALUES <
 cast('2009-01-03 00:00:00' as timestamp)
 ---- RESULTS
+'Range partition has been added.'
 ====
 ---- QUERY
 show range partitions ts_ranges
@@ -448,6 +465,7 @@ alter table ts_ranges drop range partition
 cast('2009-01-02 00:00:00' as timestamp) <= VALUES <
 cast('2009-01-03 00:00:00' as timestamp)
 ---- RESULTS
+'Range partition has been dropped.'
 ====
 ---- QUERY
 show range partitions ts_ranges

Reply via email to