You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ta...@apache.org on 2018/04/12 15:28:52 UTC

[1/3] impala git commit: IMPALA-5903: Inconsistent specification of result set and result set metadata

Repository: impala
Updated Branches:
  refs/heads/2.x bbe534298 -> 867fc82b8


http://git-wip-us.apache.org/repos/asf/impala/blob/867fc82b/testdata/workloads/functional-query/queries/QueryTest/kudu_create.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/kudu_create.test b/testdata/workloads/functional-query/queries/QueryTest/kudu_create.test
index 204fc68..394157c 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/kudu_create.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/kudu_create.test
@@ -84,6 +84,7 @@ create table tab (a int not null primary key)
 partition by range (a) (partition value = false)
 stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # Invalid hostname
@@ -107,6 +108,7 @@ Couldn't resolve this master's address bogus.host.name:7051
 create table tdata_master_addresses_whitespace (id int primary key) stored as kudu
   tblproperties('kudu.master_addresses' = '  localhost  ')
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 insert into tdata_master_addresses_whitespace values (0), (1)
@@ -120,6 +122,7 @@ create table ignore_column_case (Id int, NAME string, vAlf float, vali bigint,
   primary key (Id, NAME)) PARTITION BY RANGE (PARTITION VALUE = (1, 'Martin'))
   STORED AS KUDU
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 insert into ignore_column_case values (1, 'Martin', 1.0, 10);
@@ -145,6 +148,7 @@ create table tbl_with_null_defaults (x int primary key, i1 tinyint default null,
   valdec8 decimal(18) default null, valdec16 decimal(38) default null)
   partition by hash (x) partitions 3 stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 insert into tbl_with_null_defaults (x) values (1);
@@ -181,6 +185,7 @@ partition by range (
   partition '2009-01-03 00:00:00' <= VALUES
 ) stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 show range partitions ts_ranges
@@ -236,6 +241,7 @@ create table ts_default (i int primary key, ts1 timestamp,
                          ts2 timestamp default cast('2009-01-01 00:00:00' as timestamp))
 partition by hash(i) partitions 3 stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 insert into ts_default (i) values (1);
@@ -267,6 +273,7 @@ INT,TIMESTAMP,TIMESTAMP
 create table unpartitioned_kudu_table (col0 bigint primary key, col1 string)
 stored as kudu
 ---- RESULTS
+'Table has been created.'
 ---- ERRORS
 Unpartitioned Kudu tables are inefficient for large data sizes.
 ====
@@ -312,6 +319,7 @@ create table create_decimal
  primary key (decimal_4))
 stored as kudu;
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # Create as select table with decimal columns and primary key
@@ -327,4 +335,4 @@ select * from ctas_decimal;
 132842,333,12345.6789000000,0.12345678900000000000000000000000000000,0.77889,1
 ---- TYPES
 DECIMAL,DECIMAL,DECIMAL,DECIMAL,DECIMAL,DECIMAL
-====
\ No newline at end of file
+====

http://git-wip-us.apache.org/repos/asf/impala/blob/867fc82b/testdata/workloads/functional-query/queries/QueryTest/kudu_delete.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/kudu_delete.test b/testdata/workloads/functional-query/queries/QueryTest/kudu_delete.test
index 3abcdab..9fd7f32 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/kudu_delete.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/kudu_delete.test
@@ -8,6 +8,7 @@ create table tdata
   PARTITION BY RANGE (PARTITION VALUES < 100, PARTITION 100 <= VALUES < 1000,
   PARTITION 1000 <= VALUES <= 10000) STORED AS KUDU
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 insert into table tdata values
@@ -385,6 +386,7 @@ STRING,BIGINT,TINYINT,SMALLINT,BOOLEAN,INT,DOUBLE,FLOAT
 create table impala_3454 (key_1 tinyint, key_2 bigint, PRIMARY KEY (key_1, key_2))
   PARTITION BY HASH PARTITIONS 3 STORED AS KUDU
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 insert into impala_3454 values

http://git-wip-us.apache.org/repos/asf/impala/blob/867fc82b/testdata/workloads/functional-query/queries/QueryTest/kudu_insert.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/kudu_insert.test b/testdata/workloads/functional-query/queries/QueryTest/kudu_insert.test
index 6bba77a..da1bfac 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/kudu_insert.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/kudu_insert.test
@@ -8,6 +8,7 @@ create table tdata
   PARTITION BY RANGE (PARTITION VALUES < 10, PARTITION 10 <= VALUES < 30,
   PARTITION 30 <= VALUES) STORED AS KUDU
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # VALUES, single row, all target cols, no errors
@@ -295,6 +296,7 @@ create table allkeytypes (i1 tinyint, i2 smallint, i3 int, i4 bigint, name strin
   partition value = (2,2,2,2,'2','2009-01-01 00:02:00.100000000'),
   partition value = (3,3,3,3,'3','2009-01-01 00:03:00.300000000')) stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 insert into allkeytypes select cast(id as tinyint), smallint_col, int_col,
@@ -325,6 +327,7 @@ create table tbl_with_defaults (a int primary key, b int null default 10,
   i decimal(9, 2) default 1111.11) partition by hash (a)
   partitions 3 stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 insert into tbl_with_defaults (a, f) values (1, 1), (2, 2), (3, 3), (4, 4)
@@ -360,6 +363,7 @@ INT,INT,INT,INT,INT,INT,STRING,BOOLEAN,DECIMAL
 ---- QUERY
 alter table tbl_with_defaults add columns (j int null, k int not null default 10000)
 ---- RESULTS
+'Column has been added/replaced.'
 ====
 ---- QUERY
 select * from tbl_with_defaults
@@ -421,6 +425,7 @@ INT,INT,INT,INT,INT,INT,STRING,BOOLEAN,DECIMAL,INT,INT
 create table multiple_partition_cols (x bigint, y bigint, z string, primary key(x, y))
 partition by hash(x, y) partitions 8 stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 # SELECT with constant

http://git-wip-us.apache.org/repos/asf/impala/blob/867fc82b/testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test b/testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test
index 96cbc14..17d365b 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/kudu_partition_ddl.test
@@ -5,6 +5,7 @@ create table simple_hash (id int, name string, valf float, vali bigint,
   primary key (id, name)) partition by hash(id) partitions 4,
   hash(name) partitions 2 stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 show table stats simple_hash
@@ -29,6 +30,7 @@ create table range_part_bounds (id int, name string, valf float, vali bigint,
   (partition values <= 10, partition 10 < values <= 20, partition 20 < values)
   stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 show table stats range_part_bounds
@@ -48,6 +50,7 @@ create table range_part_single (id int, name string, valf float, vali bigint,
   (partition value = 1, partition value = 10, partition value = 100)
   stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 show table stats range_part_single
@@ -68,6 +71,7 @@ create table range_part_multiple_bounds (id int, name string, valf float,
   (partition values <= 10, partition 10 < values <= 20, partition 20 < values <= 30,
    partition value = 40, partition value = 50) stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 show table stats range_part_multiple_bounds
@@ -89,6 +93,7 @@ create table range_part_multiple_cols (id int, name string, valf float, vali big
   (partition value = (10, 'martin'), partition value = (20, 'dimitris'),
    partition value = (30, 'matthew')) stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 -- Test printing of multiple column range partitioning
@@ -116,6 +121,7 @@ create table range_part_single_string_col (id int, name string, valf float,
   (partition values <= 'aaa', partition 'aaa' < values <= 'bbb',
    partition 'bbb' < values <= 'ccc', partition value = 'ddd') stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 show table stats range_part_single_string_col
@@ -135,6 +141,7 @@ create table simple_hash_range (id int, name string, valf float, vali bigint,
   primary key (id, name)) partition by hash(id) partitions 4, range(id, name)
   (partition value = (10, 'martin'), partition value = (20, 'alex')) stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 show table stats simple_hash_range
@@ -182,6 +189,7 @@ INT,STRING,STRING,STRING,INT
 create table simple_hash_all_columns (id int, name string, valf float, vali bigint,
   primary key (id, name)) partition by hash partitions 4 stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 show table stats simple_hash_all_columns
@@ -202,6 +210,7 @@ create table simple_range_all_columns (id int, name string, valf float, vali big
   (partition value = (1, 'a'), partition value = (2, 'b'))
   stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 show table stats simple_range_all_columns
@@ -220,6 +229,7 @@ create table range_complex_const_boundary_vals (x int, y int, primary key (x))
   partition factorial(4) < values < factorial(5), partition value = factorial(6))
   stored as kudu
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 show table stats range_complex_const_boundary_vals

http://git-wip-us.apache.org/repos/asf/impala/blob/867fc82b/testdata/workloads/functional-query/queries/QueryTest/kudu_stats.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/kudu_stats.test b/testdata/workloads/functional-query/queries/QueryTest/kudu_stats.test
index 697a3a3..23a43b5 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/kudu_stats.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/kudu_stats.test
@@ -4,6 +4,7 @@ create table simple (id int primary key, name string, valf float, vali bigint)
   partition by range (partition values < 10, partition 10 <= values < 30,
   partition 30 <= values) stored as kudu tblproperties('kudu.num_tablet_replicas' = '1')
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 show table stats simple

http://git-wip-us.apache.org/repos/asf/impala/blob/867fc82b/testdata/workloads/functional-query/queries/QueryTest/kudu_update.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/kudu_update.test b/testdata/workloads/functional-query/queries/QueryTest/kudu_update.test
index 8520677..8b7759b 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/kudu_update.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/kudu_update.test
@@ -8,6 +8,7 @@ create table tdata
   PARTITION BY RANGE (PARTITION VALUES < 10, PARTITION 10 <= VALUES < 30,
   PARTITION 30 <= VALUES <= 10000) STORED AS KUDU
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 insert into tdata values

http://git-wip-us.apache.org/repos/asf/impala/blob/867fc82b/testdata/workloads/functional-query/queries/QueryTest/kudu_upsert.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/kudu_upsert.test b/testdata/workloads/functional-query/queries/QueryTest/kudu_upsert.test
index 1c12f33..5cdfef2 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/kudu_upsert.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/kudu_upsert.test
@@ -7,6 +7,7 @@ create table tdata
   PARTITION BY RANGE (PARTITION VALUES < 10, PARTITION 10 <= VALUES < 30,
   PARTITION 30 <= VALUES) STORED AS KUDU
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 insert into table tdata values

http://git-wip-us.apache.org/repos/asf/impala/blob/867fc82b/testdata/workloads/functional-query/queries/QueryTest/libs_with_same_filenames.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/libs_with_same_filenames.test b/testdata/workloads/functional-query/queries/QueryTest/libs_with_same_filenames.test
index 64fdced..71ab444 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/libs_with_same_filenames.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/libs_with_same_filenames.test
@@ -6,6 +6,7 @@ location '$FILESYSTEM_PREFIX/test-warehouse/libTestUdfs.so' symbol='NoArgs';
 create function no_args2() returns string
 location '$FILESYSTEM_PREFIX/test-warehouse/udf_test/libTestUdfs.so' symbol='NoArgs';
 ---- RESULTS
+'Function has been created.'
 ====
 ---- QUERY
 select no_args();

http://git-wip-us.apache.org/repos/asf/impala/blob/867fc82b/testdata/workloads/functional-query/queries/QueryTest/load.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/load.test b/testdata/workloads/functional-query/queries/QueryTest/load.test
index 060868f..4528380 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/load.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/load.test
@@ -3,11 +3,13 @@
 alter table functional.test_load add partition
 (year=2009, month=1)
 ---- RESULTS
+'New partition has been added to the table.'
 ====
 ---- QUERY
 alter table functional.test_load add partition
 (year=2010, month=1)
 ---- RESULTS
+'New partition has been added to the table.'
 ====
 ---- QUERY
 # Insert some data into one of the partitions, used to verify we are not clobbering

http://git-wip-us.apache.org/repos/asf/impala/blob/867fc82b/testdata/workloads/functional-query/queries/QueryTest/local-filesystem.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/local-filesystem.test b/testdata/workloads/functional-query/queries/QueryTest/local-filesystem.test
index 50e2666..1239eea 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/local-filesystem.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/local-filesystem.test
@@ -5,6 +5,7 @@ create external table tbl
 row format delimited fields terminated by ','
 location 'file://$IMPALA_HOME/testdata/data/local_tbl'
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 describe tbl

http://git-wip-us.apache.org/repos/asf/impala/blob/867fc82b/testdata/workloads/functional-query/queries/QueryTest/partition-ddl-predicates-all-fs.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/partition-ddl-predicates-all-fs.test b/testdata/workloads/functional-query/queries/QueryTest/partition-ddl-predicates-all-fs.test
index f90dcff..ea4fb0d 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/partition-ddl-predicates-all-fs.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/partition-ddl-predicates-all-fs.test
@@ -157,7 +157,7 @@ STRING, STRING, BIGINT, BIGINT, STRING, STRING, STRING, STRING, STRING, STRING
 # Tests no matching partition.
 alter table p1 partition (j=100) set location '$FILESYSTEM_PREFIX/test-warehouse/newtable';
 ---- RESULTS
-# TODO: IMPALA-6775
+'New location has been set.'
 ==== QUERY
 # Check nothing was updated.
 show partitions p1
@@ -204,4 +204,4 @@ alter table p1 partition (j=100) set row format delimited fields terminated by '
 'Updated 0 partition(s).'
 ---- TYPES
 STRING
-====
\ No newline at end of file
+====

http://git-wip-us.apache.org/repos/asf/impala/blob/867fc82b/testdata/workloads/functional-query/queries/QueryTest/test-unmatched-schema.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/test-unmatched-schema.test b/testdata/workloads/functional-query/queries/QueryTest/test-unmatched-schema.test
index 926c5bf..4605458 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/test-unmatched-schema.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/test-unmatched-schema.test
@@ -29,6 +29,7 @@ BIGINT, STRING, INT, INT
 ---- QUERY
 alter table jointbl_test add columns(new_col string)
 ---- RESULTS
+'New column(s) have been added to the table.'
 ====
 ---- QUERY
 select * from jointbl_test
@@ -58,6 +59,7 @@ BIGINT, STRING, INT, INT, STRING
 ---- QUERY
 alter table jointbl_test add columns(new_int_col int)
 ---- RESULTS
+'New column(s) have been added to the table.'
 ====
 ---- QUERY
 select * from jointbl_test
@@ -87,6 +89,7 @@ BIGINT, STRING, INT, INT, STRING, INT
 ---- QUERY
 alter table jointbl_test drop column new_int_col
 ---- RESULTS
+'Column has been dropped.'
 ====
 ---- QUERY
 select * from jointbl_test
@@ -116,10 +119,12 @@ BIGINT, STRING, INT, INT, STRING
 ---- QUERY
 alter table jointbl_test drop column new_col
 ---- RESULTS
+'Column has been dropped.'
 ====
 ---- QUERY
 alter table jointbl_test drop column alltypes_id
 ---- RESULTS
+'Column has been dropped.'
 ====
 ---- QUERY
 select * from jointbl_test

http://git-wip-us.apache.org/repos/asf/impala/blob/867fc82b/testdata/workloads/functional-query/queries/QueryTest/truncate-table.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/truncate-table.test b/testdata/workloads/functional-query/queries/QueryTest/truncate-table.test
index 6bfa21c..a8d2a80 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/truncate-table.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/truncate-table.test
@@ -166,6 +166,7 @@ STRING, STRING, BIGINT, BIGINT, BIGINT, DOUBLE
 # TRUNCATE IF EXISTS does not fail on non existent table
 truncate table if exists non_existent;
 ---- RESULTS
+'Table does not exist.'
 ====
 ---- QUERY
 # Create an unpartitioned table.
@@ -182,6 +183,7 @@ BIGINT
 # TRUNCATE IF EXISTS base scenario
 truncate table if exists t3;
 ---- RESULTS
+'Table has been truncated.'
 ====
 ---- QUERY
 # Verify that truncate was successful

http://git-wip-us.apache.org/repos/asf/impala/blob/867fc82b/testdata/workloads/functional-query/queries/QueryTest/udf-errors.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/udf-errors.test b/testdata/workloads/functional-query/queries/QueryTest/udf-errors.test
index fb91070..252d85c 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/udf-errors.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/udf-errors.test
@@ -4,6 +4,7 @@ create function if not exists hive_pi() returns double
 location '$FILESYSTEM_PREFIX/test-warehouse/hive-exec.jar'
 symbol='org.apache.hadoop.hive.ql.udf.UDFPI';
 ---- RESULTS
+'Function has been created.'
 ====
 ---- QUERY
 create function if not exists foo() returns double
@@ -33,6 +34,7 @@ create function if not exists twenty_args(int, int, int, int, int, int,
 location '$FILESYSTEM_PREFIX/test-warehouse/libTestUdfs.so'
 symbol='TwentyArgs';
 ---- RESULTS
+'Function has been created.'
 ====
 ---- QUERY
 # Verifies that interpretation can support up to 20 arguments
@@ -49,6 +51,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
 # Verifies that interpretation fails with more than 20 arguments.
@@ -63,6 +66,7 @@ create function if not exists nine_args_ir(int, int, int, int, int, int,
 location '$FILESYSTEM_PREFIX/test-warehouse/test-udfs.ll'
 symbol='NineArgs';
 ---- RESULTS
+'Function has been created.'
 ====
 ---- QUERY
 select nine_args_ir(1,2,3,4,5,6,7,8,9);
@@ -74,12 +78,14 @@ create function if not exists bad_expr(double) returns boolean
 location '$FILESYSTEM_PREFIX/test-warehouse/libTestUdfs.so'
 symbol='BadExpr' prepare_fn='BadExprPrepare' close_fn='BadExprClose';
 ---- RESULTS
+'Function has been created.'
 ====
 ---- QUERY
 create function if not exists bad_expr2(double) returns boolean
 location '$FILESYSTEM_PREFIX/test-warehouse/libTestUdfs.so'
 symbol='BadExpr' prepare_fn='BadExpr2Prepare' close_fn='BadExprClose';
 ---- RESULTS
+'Function has been created.'
 ====
 ---- QUERY
 select count(t1.int_col) from functional.alltypes t1 join functional.alltypes t2
@@ -120,4 +126,5 @@ drop function nine_args_ir(int, int, int, int, int, int, int, int, int);
 drop function bad_expr(double);
 drop function bad_expr2(double);
 ---- RESULTS
+'Function has been dropped.'
 ====

http://git-wip-us.apache.org/repos/asf/impala/blob/867fc82b/testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/views-ddl.test b/testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
index 98f116f..0461222 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
@@ -4,12 +4,14 @@
 create view $DATABASE.simple_view as
 select * from functional.alltypes
 ---- RESULTS
+'View has been created.'
 ====
 ---- QUERY
 # Test that 'if not exists' swallows the error (view already exists)
 create view if not exists $DATABASE.simple_view as
 select * from functional.alltypesagg
 ---- RESULTS
+'View already exists.'
 ====
 ---- QUERY
 # Create another simple view with 'if not exists' on a subset of
@@ -18,12 +20,14 @@ create view if not exists
 $DATABASE.simple_view_sub (x, y comment 'hello', z) as
 select int_col, string_col, timestamp_col from functional.alltypes
 ---- RESULTS
+'View has been created.'
 ====
 ---- QUERY
 # Create a view on a parquet table (Hive cannot create/read/write parquet)
 create view $DATABASE.parquet_view as
 select * from functional_parquet.alltypes where id < 20
 ---- RESULTS
+'View has been created.'
 ====
 ---- QUERY
 # Create a complex view with predicates, joins, aggregates and order by
@@ -34,12 +38,14 @@ on a.id = b.id where a.bigint_col < 50
 group by b.string_col having count(a.bigint_col) > 1
 order by b.string_col limit 100
 ---- RESULTS
+'View has been created.'
 ====
 ---- QUERY
 # Create a view on a view
 create view $DATABASE.view_view (aaa, bbb) as
 select * from $DATABASE.complex_view
 ---- RESULTS
+'View has been created.'
 ====
 ---- QUERY
 # Test that the views are displayed by 'show tables'
@@ -155,6 +161,7 @@ bigint
 # Test dropping a view
 drop view $DATABASE.simple_view_sub
 ---- RESULTS
+'View has been dropped.'
 ====
 ---- QUERY
 # Test that the view is gone
@@ -169,15 +176,18 @@ show tables in $DATABASE
 # Test 'if exists' for dropping a view (view does not exist)
 drop view if exists $DATABASE.bad_view
 ---- RESULTS
+'View does not exist.'
 ====
 ---- QUERY
 # Test 'if exists' does not drop a table with same name
 create table $DATABASE.drop_tbl_test(a int)
 ---- RESULTS
+'Table has been created.'
 ====
 ---- QUERY
 drop view if exists $DATABASE.drop_tbl_test
 ---- RESULTS
+'Drop view is not allowed on a table.'
 ====
 ---- QUERY
 # Test drop table 'if exists' does not drop a view with same name.
@@ -185,6 +195,7 @@ drop view if exists $DATABASE.drop_tbl_test
 # still be listed in the subsequent show tables output (as a view).
 drop table if exists $DATABASE.complex_view
 ---- RESULTS
+'Drop table is not allowed on a view.'
 ====
 ---- QUERY
 # Test that the table is present
@@ -200,11 +211,13 @@ show tables in $DATABASE
 # Test renaming a view
 alter view $DATABASE.view_view rename to $DATABASE.view_on_view
 ---- RESULTS
+'Renaming was successful.'
 ====
 ---- QUERY
 # Test renaming a parquet view
 alter view $DATABASE.parquet_view rename to $DATABASE.new_parquet_view
 ---- RESULTS
+'Renaming was successful.'
 ====
 ---- QUERY
 # Test that the view was renamed
@@ -222,6 +235,7 @@ alter view $DATABASE.new_parquet_view as
 select bigint_col, string_col from functional_parquet.alltypesagg
 where bigint_col is null limit 10
 ---- RESULTS
+'View has been altered.'
 ====
 ---- QUERY
 # Test querying the altered view
@@ -236,6 +250,7 @@ bigint,bigint
 create view $DATABASE.const_view
 as select 1, 'a', cast(10.0 as float)
 ---- RESULTS
+'View has been created.'
 ====
 ---- QUERY
 select * from $DATABASE.const_view
@@ -251,6 +266,7 @@ create view $DATABASE.paren_view as
 select count(*) from functional.alltypessmall
 where true and (true or false) and false
 ---- RESULTS
+'View has been created.'
 ====
 ---- QUERY
 # Test that parentheses are preserved in view creation.
@@ -265,6 +281,7 @@ bigint
 create view $DATABASE.decimal_view as
 select * from functional.decimal_tbl
 ---- RESULTS
+'View has been created.'
 ====
 ---- QUERY
 # Query a view with decimal columns. Regression test for IMPALA-1021.


[3/3] impala git commit: IMPALA-6215: Removes race when using LibCache.

Posted by ta...@apache.org.
IMPALA-6215: Removes race when using LibCache.

Re-do the previously reverted change for IMPALA-6215. This patch
addresses the flakes listed in IMPALA-6092, which have become more
urgent recently.

LibCache's api to provide access to locally cached files has a race.
Currently, the client of the cache accesses the locally cached path
as a string, but nothing guarantees that the associated file is not
removed before the client is done using it. This race is suspected
as the root cause for the flakiness seen in IMPALA-6092. These tests
fail once in a while with classloader errors unable to load java udf
classes. In these tests, the lib cache makes no guarantee that the path
to the jar will remain valid from the time the path is acquired through
the time needed to fetch the jar and resolve the needed classes.

LibCache offers liveness guarantees for shared objects via reference
counting. The fix in this patch extends this API to also cover paths
to locally cached files.

This fix *only* addresses the path race. General cleanup of the api will
be done separately.

Testing:
   - added a test to test_udfs.py that does many concurrent udf uses and
     removals. By increasing the concurrent operations to 100, the issue
     in IMPALA-6092 is locally reproducible on every run. With this fix,
     the problem is no longer reproducible with this test.

Change-Id: I72ac0dfb13cf37d79e25c5b8a258b65f2dad097f
Reviewed-on: http://gerrit.cloudera.org:8080/9968
Reviewed-by: Vuk Ercegovac <ve...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/2.x
Commit: 226b914dd02dba4aa389ed7b4c0023ddfd229096
Parents: bbe5342
Author: Vuk Ercegovac <ve...@cloudera.com>
Authored: Tue Nov 21 08:41:03 2017 -0800
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Thu Apr 12 03:00:07 2018 +0000

----------------------------------------------------------------------
 be/src/codegen/llvm-codegen.cc               |  5 +-
 be/src/exec/external-data-source-executor.cc |  6 +-
 be/src/exprs/hive-udf-call.cc                | 57 +++++++--------
 be/src/exprs/hive-udf-call.h                 |  3 -
 be/src/runtime/lib-cache.cc                  | 18 +++--
 be/src/runtime/lib-cache.h                   | 39 +++++++++--
 be/src/service/fe-support.cc                 | 10 +--
 tests/query_test/test_udfs.py                | 84 +++++++++++++++++++++--
 8 files changed, 167 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/226b914d/be/src/codegen/llvm-codegen.cc
----------------------------------------------------------------------
diff --git a/be/src/codegen/llvm-codegen.cc b/be/src/codegen/llvm-codegen.cc
index 5ac19ad..c8fd8eb 100644
--- a/be/src/codegen/llvm-codegen.cc
+++ b/be/src/codegen/llvm-codegen.cc
@@ -339,9 +339,10 @@ Status LlvmCodeGen::LinkModuleFromLocalFs(const string& file) {
 
 Status LlvmCodeGen::LinkModuleFromHdfs(const string& hdfs_location, const time_t mtime) {
   if (linked_modules_.find(hdfs_location) != linked_modules_.end()) return Status::OK();
+  LibCacheEntryHandle handle;
   string local_path;
-  RETURN_IF_ERROR(LibCache::instance()->GetLocalLibPath(
-      hdfs_location, LibCache::TYPE_IR, mtime, &local_path));
+  RETURN_IF_ERROR(LibCache::instance()->GetLocalPath(
+      hdfs_location, LibCache::TYPE_IR, mtime, &handle, &local_path));
   RETURN_IF_ERROR(LinkModuleFromLocalFs(local_path));
   linked_modules_.insert(hdfs_location);
   return Status::OK();

http://git-wip-us.apache.org/repos/asf/impala/blob/226b914d/be/src/exec/external-data-source-executor.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/external-data-source-executor.cc b/be/src/exec/external-data-source-executor.cc
index e8217b7..5c06d3a 100644
--- a/be/src/exec/external-data-source-executor.cc
+++ b/be/src/exec/external-data-source-executor.cc
@@ -136,12 +136,12 @@ ExternalDataSourceExecutor::~ExternalDataSourceExecutor() {
 Status ExternalDataSourceExecutor::Init(const string& jar_path,
     const string& class_name, const string& api_version, const string& init_string) {
   DCHECK(!is_initialized_);
+  LibCacheEntryHandle handle;
   string local_jar_path;
   // TODO(IMPALA-6727): pass the mtime from the coordinator. for now, skip the mtime
   // check (-1).
-  RETURN_IF_ERROR(LibCache::instance()->GetLocalLibPath(
-      jar_path, LibCache::TYPE_JAR, -1, &local_jar_path));
-
+  RETURN_IF_ERROR(LibCache::instance()->GetLocalPath(
+      jar_path, LibCache::TYPE_JAR, -1, &handle, &local_jar_path));
   JNIEnv* jni_env = getJNIEnv();
 
   // Add a scoped cleanup jni reference object. This cleans up local refs made below.

http://git-wip-us.apache.org/repos/asf/impala/blob/226b914d/be/src/exprs/hive-udf-call.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/hive-udf-call.cc b/be/src/exprs/hive-udf-call.cc
index be3965a..b7715b7 100644
--- a/be/src/exprs/hive-udf-call.cc
+++ b/be/src/exprs/hive-udf-call.cc
@@ -174,10 +174,6 @@ Status HiveUdfCall::Init(const RowDescriptor& row_desc, RuntimeState* state) {
   // Initialize children first.
   RETURN_IF_ERROR(ScalarExpr::Init(row_desc, state));
 
-  // Copy the Hive Jar from hdfs to local file system.
-  RETURN_IF_ERROR(LibCache::instance()->GetLocalLibPath(
-      fn_.hdfs_location, LibCache::TYPE_JAR, fn_.last_modified_time, &local_location_));
-
   // Initialize input_byte_offsets_ and input_buffer_size_
   for (int i = 0; i < GetNumChildren(); ++i) {
     input_byte_offsets_.push_back(input_buffer_size_);
@@ -202,30 +198,35 @@ Status HiveUdfCall::OpenEvaluator(FunctionContext::FunctionStateScope scope,
   JNIEnv* env = getJNIEnv();
   if (env == NULL) return Status("Failed to get/create JVM");
 
-  THiveUdfExecutorCtorParams ctor_params;
-  ctor_params.fn = fn_;
-  ctor_params.local_location = local_location_;
-  ctor_params.input_byte_offsets = input_byte_offsets_;
-
-  jni_ctx->input_values_buffer = new uint8_t[input_buffer_size_];
-  jni_ctx->input_nulls_buffer = new uint8_t[GetNumChildren()];
-  jni_ctx->output_value_buffer = new uint8_t[type().GetSlotSize()];
-
-  ctor_params.input_buffer_ptr = (int64_t)jni_ctx->input_values_buffer;
-  ctor_params.input_nulls_ptr = (int64_t)jni_ctx->input_nulls_buffer;
-  ctor_params.output_buffer_ptr = (int64_t)jni_ctx->output_value_buffer;
-  ctor_params.output_null_ptr = (int64_t)&jni_ctx->output_null_value;
-
-  jbyteArray ctor_params_bytes;
-
-  // Add a scoped cleanup jni reference object. This cleans up local refs made
-  // below.
-  JniLocalFrame jni_frame;
-  RETURN_IF_ERROR(jni_frame.push(env));
-
-  RETURN_IF_ERROR(SerializeThriftMsg(env, &ctor_params, &ctor_params_bytes));
-  // Create the java executor object
-  jni_ctx->executor = env->NewObject(executor_cl_, executor_ctor_id_, ctor_params_bytes);
+  {
+    LibCacheEntryHandle handle;
+    string local_location;
+    RETURN_IF_ERROR(LibCache::instance()->GetLocalPath(fn_.hdfs_location,
+        LibCache::TYPE_JAR, fn_.last_modified_time, &handle, &local_location));
+    THiveUdfExecutorCtorParams ctor_params;
+    ctor_params.fn = fn_;
+    ctor_params.local_location = local_location;
+    ctor_params.input_byte_offsets = input_byte_offsets_;
+
+    jni_ctx->input_values_buffer = new uint8_t[input_buffer_size_];
+    jni_ctx->input_nulls_buffer = new uint8_t[GetNumChildren()];
+    jni_ctx->output_value_buffer = new uint8_t[type().GetSlotSize()];
+
+    ctor_params.input_buffer_ptr = (int64_t)jni_ctx->input_values_buffer;
+    ctor_params.input_nulls_ptr = (int64_t)jni_ctx->input_nulls_buffer;
+    ctor_params.output_buffer_ptr = (int64_t)jni_ctx->output_value_buffer;
+    ctor_params.output_null_ptr = (int64_t)&jni_ctx->output_null_value;
+
+    jbyteArray ctor_params_bytes;
+
+    // Add a scoped cleanup jni reference object. This cleans up local refs made below.
+    JniLocalFrame jni_frame;
+    RETURN_IF_ERROR(jni_frame.push(env));
+
+    RETURN_IF_ERROR(SerializeThriftMsg(env, &ctor_params, &ctor_params_bytes));
+    // Create the java executor object
+    jni_ctx->executor = env->NewObject(executor_cl_, executor_ctor_id_, ctor_params_bytes);
+  }
   RETURN_ERROR_IF_EXC(env);
   RETURN_IF_ERROR(JniUtil::LocalToGlobalRef(env, jni_ctx->executor, &jni_ctx->executor));
 

http://git-wip-us.apache.org/repos/asf/impala/blob/226b914d/be/src/exprs/hive-udf-call.h
----------------------------------------------------------------------
diff --git a/be/src/exprs/hive-udf-call.h b/be/src/exprs/hive-udf-call.h
index 7ce5eb0..8ca0372 100644
--- a/be/src/exprs/hive-udf-call.h
+++ b/be/src/exprs/hive-udf-call.h
@@ -116,9 +116,6 @@ class HiveUdfCall : public ScalarExpr {
   /// error.
   AnyVal* Evaluate(ScalarExprEvaluator* eval, const TupleRow* row) const;
 
-  /// The path on the local FS to the UDF's jar
-  std::string local_location_;
-
   /// input_byte_offsets_[i] is the byte offset child ith's input argument should
   /// be written to.
   std::vector<int> input_byte_offsets_;

http://git-wip-us.apache.org/repos/asf/impala/blob/226b914d/be/src/runtime/lib-cache.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/lib-cache.cc b/be/src/runtime/lib-cache.cc
index 83bb4dc..cb1bc74 100644
--- a/be/src/runtime/lib-cache.cc
+++ b/be/src/runtime/lib-cache.cc
@@ -131,6 +131,10 @@ LibCacheEntry::~LibCacheEntry() {
   unlink(local_path.c_str());
 }
 
+LibCacheEntryHandle::~LibCacheEntryHandle() {
+  if (entry_ != nullptr) LibCache::instance()->DecrementUseCount(entry_);
+}
+
 Status LibCache::GetSoFunctionPtr(const string& hdfs_lib_file, const string& symbol,
     time_t exp_mtime, void** fn_ptr, LibCacheEntry** ent, bool quiet) {
   if (hdfs_lib_file.empty()) {
@@ -181,14 +185,16 @@ void LibCache::DecrementUseCount(LibCacheEntry* entry) {
   if (can_delete) delete entry;
 }
 
-Status LibCache::GetLocalLibPath(
-    const string& hdfs_lib_file, LibType type, time_t exp_mtime, string* local_path) {
-  unique_lock<mutex> lock;
+Status LibCache::GetLocalPath(const std::string& hdfs_lib_file, LibType type,
+    time_t exp_mtime, LibCacheEntryHandle* handle, string* path) {
+  DCHECK(handle != nullptr && handle->entry() == nullptr);
   LibCacheEntry* entry = nullptr;
+  unique_lock<mutex> lock;
   RETURN_IF_ERROR(GetCacheEntry(hdfs_lib_file, type, exp_mtime, &lock, &entry));
   DCHECK(entry != nullptr);
-  DCHECK_EQ(entry->type, type);
-  *local_path = entry->local_path;
+  ++entry->use_count;
+  handle->SetEntry(entry);
+  *path = entry->local_path;
   return Status::OK();
 }
 
@@ -422,7 +428,7 @@ Status LibCache::RefreshCacheEntry(const string& hdfs_lib_file, LibType type,
 
     // Let the caller propagate any error that occurred when loading the entry.
     RETURN_IF_ERROR((*entry)->copy_file_status);
-    DCHECK_EQ((*entry)->type, type);
+    DCHECK_EQ((*entry)->type, type) << (*entry)->local_path;
     DCHECK(!(*entry)->local_path.empty());
   }
   return Status::OK();

http://git-wip-us.apache.org/repos/asf/impala/blob/226b914d/be/src/runtime/lib-cache.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/lib-cache.h b/be/src/runtime/lib-cache.h
index 820a1a8..b8a2981 100644
--- a/be/src/runtime/lib-cache.h
+++ b/be/src/runtime/lib-cache.h
@@ -49,11 +49,16 @@ class RuntimeState;
 /// using the library. When the caller requests a ptr into the library, they
 /// are given the entry handle and must decrement the ref count when they
 /// are done.
+/// Note: Explicitly managing this reference count at the client is error-prone. See the
+/// api for accessing a path, GetLocalPath(), that uses the handle's scope to manage the
+/// reference count.
 //
 /// TODO:
 /// - refresh libraries
-/// - better cached module management.
+/// - better cached module management
+/// - improve the api to be less error-prone (IMPALA-6439)
 struct LibCacheEntry;
+class LibCacheEntryHandle;
 
 class LibCache {
  public:
@@ -71,15 +76,19 @@ class LibCache {
   /// Initializes the libcache. Must be called before any other APIs.
   static Status Init();
 
-  /// Gets the local file system path for the library at 'hdfs_lib_file'. If
+  /// Gets the local 'path' used to cache the file stored at the global 'hdfs_lib_file'. If
   /// this file is not already on the local fs, or if the cached entry's last modified
   /// is older than expected mtime, 'exp_mtime', it copies it and caches the result.
   /// An 'exp_mtime' of -1 makes the mtime check a no-op.
+  ///
+  /// 'handle' must remain in scope while 'path' is used. The reference count to the
+  /// underlying cache entry is decremented when 'handle' goes out-of-scope.
+  ///
   /// Returns an error if 'hdfs_lib_file' cannot be copied to the local fs or if
   /// exp_mtime differs from the mtime on the file system.
   /// If error is due to refresh, then the entry will be removed from the cache.
-  Status GetLocalLibPath(const std::string& hdfs_lib_file, LibType type, time_t exp_mtime,
-      std::string* local_path);
+  Status GetLocalPath(const std::string& hdfs_lib_file, LibType type, time_t exp_mtime,
+      LibCacheEntryHandle* handle, string* path);
 
   /// Returns status.ok() if the symbol exists in 'hdfs_lib_file', non-ok otherwise.
   /// If status.ok() is true, 'mtime' is set to the cache entry's last modified time.
@@ -107,6 +116,7 @@ class LibCache {
   /// An 'exp_mtime' of -1 makes the mtime check a no-op.
   /// An error is returned if exp_mtime differs from the mtime on the file system.
   /// If error is due to refresh, then the entry will be removed from the cache.
+  /// TODO: api is error-prone. upgrade to LibCacheEntryHandle (see IMPALA-6439).
   Status GetSoFunctionPtr(const std::string& hdfs_lib_file, const std::string& symbol,
       time_t exp_mtime, void** fn_ptr, LibCacheEntry** entry, bool quiet = false);
 
@@ -204,6 +214,27 @@ class LibCache {
       const std::string& hdfs_lib_file, const LibMap::iterator& entry_iterator);
 };
 
+/// Handle for a LibCacheEntry that decrements its reference count when the handle is
+/// destroyed or re-used for another entry.
+class LibCacheEntryHandle {
+ public:
+  LibCacheEntryHandle() {}
+  ~LibCacheEntryHandle();
+
+ private:
+  friend class LibCache;
+
+  LibCacheEntry* entry() const { return entry_; }
+  void SetEntry(LibCacheEntry* entry) {
+    if (entry_ != nullptr) LibCache::instance()->DecrementUseCount(entry);
+    entry_ = entry;
+  }
+
+  LibCacheEntry* entry_ = nullptr;
+
+  DISALLOW_COPY_AND_ASSIGN(LibCacheEntryHandle);
+};
+
 }
 
 #endif

http://git-wip-us.apache.org/repos/asf/impala/blob/226b914d/be/src/service/fe-support.cc
----------------------------------------------------------------------
diff --git a/be/src/service/fe-support.cc b/be/src/service/fe-support.cc
index a8906a0..187d14e 100644
--- a/be/src/service/fe-support.cc
+++ b/be/src/service/fe-support.cc
@@ -300,9 +300,10 @@ static void ResolveSymbolLookup(const TSymbolLookupParams params,
       // Refresh the library if necessary.
       LibCache::instance()->SetNeedsRefresh(params.location);
     }
+    LibCacheEntryHandle handle;
     string dummy_local_path;
-    Status status = LibCache::instance()->GetLocalLibPath(
-        params.location, type, -1, &dummy_local_path);
+    Status status = LibCache::instance()->GetLocalPath(
+        params.location, type, -1, &handle, &dummy_local_path);
     if (!status.ok()) {
       result->__set_result_code(TSymbolLookupResultCode::BINARY_NOT_FOUND);
       result->__set_error_msg(status.GetDetail());
@@ -397,10 +398,11 @@ Java_org_apache_impala_service_FeSupport_NativeCacheJar(
       JniUtil::internal_exc_class(), nullptr);
 
   TCacheJarResult result;
+  LibCacheEntryHandle handle;
   string local_path;
   // TODO(IMPALA-6727): used for external data sources; add proper mtime.
-  Status status = LibCache::instance()->GetLocalLibPath(
-      params.hdfs_location, LibCache::TYPE_JAR, -1, &local_path);
+  Status status = LibCache::instance()->GetLocalPath(
+      params.hdfs_location, LibCache::TYPE_JAR, -1, &handle, &local_path);
   status.ToThrift(&result.status);
   if (status.ok()) result.__set_local_path(local_path);
 

http://git-wip-us.apache.org/repos/asf/impala/blob/226b914d/tests/query_test/test_udfs.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_udfs.py b/tests/query_test/test_udfs.py
index 644d80f..dc5491b 100644
--- a/tests/query_test/test_udfs.py
+++ b/tests/query_test/test_udfs.py
@@ -394,8 +394,6 @@ class TestUdfExecution(TestUdfBase):
       self.run_test_case('QueryTest/udf-non-deterministic', vector,
           use_db=unique_database)
 
-  # Runs serially as a temporary workaround for IMPALA_6092.
-  @pytest.mark.execute_serially
   def test_java_udfs(self, vector, unique_database):
     self.run_test_case('QueryTest/load-java-udfs', vector, use_db=unique_database)
     self.run_test_case('QueryTest/java-udf', vector, use_db=unique_database)
@@ -507,9 +505,6 @@ class TestUdfTargeted(TestUdfBase):
             unique_database, tgt_udf_path))
     query = "select `{0}`.fn_invalid_symbol('test')".format(unique_database)
 
-    # Dropping the function can interact with other tests whose Java classes are in
-    # the same jar. Use a copy of the jar to avoid unintended interactions.
-    # See IMPALA-6215 and IMPALA-6092 for examples.
     check_call(["hadoop", "fs", "-put", "-f", src_udf_path, tgt_udf_path])
     self.client.execute(drop_fn_stmt)
     self.client.execute(create_fn_stmt)
@@ -518,6 +513,85 @@ class TestUdfTargeted(TestUdfBase):
       assert "Unable to find class" in str(ex)
     self.client.execute(drop_fn_stmt)
 
+  def test_concurrent_jar_drop_use(self, vector, unique_database):
+    """IMPALA-6215: race between dropping/using java udf's defined in the same jar.
+       This test runs concurrent drop/use threads that result in class not found
+       exceptions when the race is present.
+    """
+    udf_src_path = os.path.join(
+      os.environ['IMPALA_HOME'], "testdata/udfs/impala-hive-udfs.jar")
+    udf_tgt_path = get_fs_path(
+      '/test-warehouse/{0}.db/impala-hive-udfs.jar'.format(unique_database))
+
+    create_fn_to_drop = """create function {0}.foo_{1}() returns string
+                           LOCATION '{2}' SYMBOL='org.apache.impala.TestUpdateUdf'"""
+    create_fn_to_use = """create function {0}.use_it(string) returns string
+                          LOCATION '{1}' SYMBOL='org.apache.impala.TestUdf'"""
+    drop_fn = "drop function if exists {0}.foo_{1}()"
+    use_fn = """select * from (select max(int_col) from functional.alltypesagg
+                where {0}.use_it(string_col) = 'blah' union all
+                (select max(int_col) from functional.alltypesagg
+                 where {0}.use_it(String_col) > '1' union all
+                (select max(int_col) from functional.alltypesagg
+                 where {0}.use_it(string_col) > '1'))) v"""
+    num_drops = 100
+    num_uses = 100
+
+    # use a unique jar for this test to avoid interactions with other tests
+    # that use the same jar
+    check_call(["hadoop", "fs", "-put", "-f", udf_src_path, udf_tgt_path])
+
+    # create all the functions.
+    setup_client = self.create_impala_client()
+    try:
+      s = create_fn_to_use.format(unique_database, udf_tgt_path)
+      setup_client.execute(s)
+    except Exception as e:
+      print e
+      assert False
+    for i in range(0, num_drops):
+      try:
+        setup_client.execute(create_fn_to_drop.format(unique_database, i, udf_tgt_path))
+      except Exception as e:
+        print e
+        assert False
+
+    errors = []
+    def use_fn_method():
+      time.sleep(5 + random.random())
+      client = self.create_impala_client()
+      try:
+        client.execute(use_fn.format(unique_database))
+      except Exception as e: errors.append(e)
+
+    def drop_fn_method(i):
+      time.sleep(1 + random.random())
+      client = self.create_impala_client()
+      try:
+        client.execute(drop_fn.format(unique_database, i))
+      except Exception as e: errors.append(e)
+
+    # create threads to use functions.
+    runner_threads = []
+    for i in range(0, num_uses):
+      runner_threads.append(threading.Thread(target=use_fn_method))
+
+    # create threads to drop functions.
+    drop_threads = []
+    for i in range(0, num_drops):
+      runner_threads.append(threading.Thread(target=drop_fn_method, args=(i, )))
+
+    # launch all runner threads.
+    for t in runner_threads: t.start()
+
+    # join all threads.
+    for t in runner_threads: t.join();
+
+    # Check for any errors.
+    for e in errors: print e
+    assert len(errors) == 0
+
+
   @SkipIfLocal.multiple_impalad
   def test_hive_udfs_missing_jar(self, vector, unique_database):
     """ IMPALA-2365: Impalad shouldn't crash if the udf jar isn't present


[2/3] impala git commit: IMPALA-5903: Inconsistent specification of result set and result set metadata

Posted by ta...@apache.org.
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 <al...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/2.x
Commit: 867fc82b8e72e8daca094d5ec3a2134c71a26210
Parents: 226b914
Author: Zoltan Borok-Nagy <bo...@cloudera.com>
Authored: Thu Apr 5 14:54:27 2018 +0200
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Thu Apr 12 03:00:07 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/867fc82b/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 e5d5e13..6d38521 100644
--- a/be/src/service/client-request-state.cc
+++ b/be/src/service/client-request-state.cc
@@ -515,6 +515,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();
   }
@@ -645,6 +647,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_.
@@ -680,7 +685,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/867fc82b/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/867fc82b/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 41395d6..556d6ac 100644
--- a/fe/src/main/java/org/apache/impala/service/Frontend.java
+++ b/fe/src/main/java/org/apache/impala/service/Frontend.java
@@ -333,21 +333,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();
@@ -355,29 +352,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();
@@ -385,7 +377,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();
@@ -393,7 +384,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();
@@ -401,21 +391,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();
@@ -423,7 +410,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();
@@ -431,7 +417,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();
@@ -439,7 +424,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();
@@ -447,7 +431,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();
@@ -455,7 +438,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();
@@ -498,7 +480,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();
@@ -507,7 +488,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();
@@ -517,10 +497,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/867fc82b/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/867fc82b/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/867fc82b/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/867fc82b/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/867fc82b/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/867fc82b/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/867fc82b/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/867fc82b/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/867fc82b/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/867fc82b/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/867fc82b/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/867fc82b/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/867fc82b/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 22d7122..856180e 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/867fc82b/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/867fc82b/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/867fc82b/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/867fc82b/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/867fc82b/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/867fc82b/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