You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2018/07/03 17:47:32 UTC

[04/46] hive git commit: HIVE-19812: Disable external table replication by default via a configuration property (Mahesh Kumar Behera, reviewed by Sankar Hariappan)

HIVE-19812: Disable external table replication by default via a configuration property (Mahesh Kumar Behera, reviewed by Sankar Hariappan)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/9bc90f28
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/9bc90f28
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/9bc90f28

Branch: refs/heads/master-txnstats
Commit: 9bc90f28cf4406d74f9597837c2b72c809b582e6
Parents: b9bac8e
Author: Sankar Hariappan <sa...@apache.org>
Authored: Mon Jul 2 16:02:45 2018 +0530
Committer: Sankar Hariappan <sa...@apache.org>
Committed: Mon Jul 2 16:02:45 2018 +0530

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |  8 +-
 .../hadoop/hive/ql/parse/TestExportImport.java  | 55 ++++++++++++++
 ...TestReplicationScenariosAcrossInstances.java | 77 ++++++++++++++++++++
 .../org/apache/hadoop/hive/ql/ErrorMsg.java     |  4 +-
 .../apache/hadoop/hive/ql/exec/ExportTask.java  |  2 +-
 .../hadoop/hive/ql/parse/repl/dump/Utils.java   |  6 ++
 .../queries/clientpositive/repl_2_exim_basic.q  |  1 +
 .../exim_03_nonpart_noncompat_colschema.q.out   |  2 +-
 .../exim_04_nonpart_noncompat_colnumber.q.out   |  2 +-
 .../exim_05_nonpart_noncompat_coltype.q.out     |  2 +-
 .../exim_06_nonpart_noncompat_storage.q.out     |  2 +-
 .../exim_07_nonpart_noncompat_ifof.q.out        |  2 +-
 .../exim_08_nonpart_noncompat_serde.q.out       |  2 +-
 .../exim_09_nonpart_noncompat_serdeparam.q.out  |  2 +-
 .../exim_10_nonpart_noncompat_bucketing.q.out   |  2 +-
 .../exim_11_nonpart_noncompat_sorting.q.out     |  2 +-
 .../exim_12_nonnative_export.q.out              |  2 +-
 .../exim_13_nonnative_import.q.out              |  2 +-
 .../clientnegative/exim_14_nonpart_part.q.out   |  2 +-
 .../clientnegative/exim_15_part_nonpart.q.out   |  2 +-
 .../exim_16_part_noncompat_schema.q.out         |  2 +-
 .../exim_19_external_over_existing.q.out        |  2 +-
 .../exim_21_part_managed_external.q.out         |  2 +-
 23 files changed, 165 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 35aee20..a3dd53e 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -466,7 +466,8 @@ public class HiveConf extends Configuration {
         new TimeValidator(TimeUnit.DAYS),
         "TTL of dump dirs before cleanup."),
     REPL_DUMP_METADATA_ONLY("hive.repl.dump.metadata.only", false,
-        "Indicates whether replication dump only metadata information or data + metadata."),
+        "Indicates whether replication dump only metadata information or data + metadata. \n"
+          + "This config makes hive.repl.include.external.tables config ineffective."),
     REPL_DUMP_INCLUDE_ACID_TABLES("hive.repl.dump.include.acid.tables", false,
         "Indicates if repl dump should include information about ACID tables. It should be \n"
             + "used in conjunction with 'hive.repl.dump.metadata.only' to enable copying of \n"
@@ -482,6 +483,11 @@ public class HiveConf extends Configuration {
     REPL_ADD_RAW_RESERVED_NAMESPACE("hive.repl.add.raw.reserved.namespace", false,
         "For TDE with same encryption keys on source and target, allow Distcp super user to access \n"
             + "the raw bytes from filesystem without decrypting on source and then encrypting on target."),
+    REPL_INCLUDE_EXTERNAL_TABLES("hive.repl.include.external.tables", false,
+        "Indicates if repl dump should include information about external tables. It should be \n"
+          + "used in conjunction with 'hive.repl.dump.metadata.only' set to false. if 'hive.repl.dump.metadata.only' \n"
+          + " is set to true then this config parameter has no effect as external table meta data is flushed \n"
+          + " always by default."),
     LOCALSCRATCHDIR("hive.exec.local.scratchdir",
         "${system:java.io.tmpdir}" + File.separator + "${system:user.name}",
         "Local scratch space for Hive jobs"),

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestExportImport.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestExportImport.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestExportImport.java
index 53d13d8..e442528 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestExportImport.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestExportImport.java
@@ -42,6 +42,7 @@ public class TestExportImport {
   protected static final Logger LOG = LoggerFactory.getLogger(TestExportImport.class);
   private static WarehouseInstance srcHiveWarehouse;
   private static WarehouseInstance destHiveWarehouse;
+  private static WarehouseInstance dumpExternalWarehouse;
 
   @Rule
   public final TestName testName = new TestName();
@@ -58,9 +59,15 @@ public class TestExportImport {
     HashMap<String, String> overridesForHiveConf = new HashMap<String, String>() {{
       put(HiveConf.ConfVars.HIVE_IN_TEST.varname, "false");
     }};
+    HashMap<String, String> overridesForHiveConfDump = new HashMap<String, String>() {{
+        put(HiveConf.ConfVars.HIVE_IN_TEST.varname, "false");
+        put(HiveConf.ConfVars.REPL_INCLUDE_EXTERNAL_TABLES.varname, "true");
+    }};
     srcHiveWarehouse =
         new WarehouseInstance(LOG, miniDFSCluster, overridesForHiveConf);
     destHiveWarehouse = new WarehouseInstance(LOG, miniDFSCluster, overridesForHiveConf);
+    dumpExternalWarehouse =
+            new WarehouseInstance(LOG, miniDFSCluster, overridesForHiveConfDump);
   }
 
   @AfterClass
@@ -110,6 +117,54 @@ public class TestExportImport {
   }
 
   @Test
+  public void testExportExternalTableSetFalse() throws Throwable {
+    String path = "hdfs:///tmp/" + dbName + "/";
+    String exportMDPath = "'" + path + "1/'";
+    String exportDataPath = "'" + path + "2/'";
+    String exportDataPathRepl = "'" + path + "3/'";
+    srcHiveWarehouse.run("create external table " + dbName + ".t1 (i int)")
+            .run("insert into table " + dbName + ".t1 values (1),(2)")
+            .run("export table " + dbName + ".t1 to " + exportMDPath + " for metadata replication('1')")
+            .run("export table " + dbName + ".t1 to " + exportDataPath)
+            .runFailure("export table " + dbName + ".t1 to " + exportDataPathRepl + " for replication('2')");
+
+    destHiveWarehouse.run("use " + replDbName)
+            .run("import table " + replDbName + ".t1 from " + exportMDPath)
+            .run("show tables like 't1'")
+            .verifyResult("t1")
+            .run("import table " + replDbName + ".t2 from " + exportDataPath)
+            .run("select * from " + replDbName + ".t2")
+            .verifyResults(new String[] {"1", "2" })
+            .runFailure("import table " + replDbName + ".t3 from " + exportDataPathRepl)
+            .run("show tables like 't3'")
+            .verifyFailure(new String[] {"t3"});
+  }
+
+  @Test
+  public void testExportExternalTableSetTrue() throws Throwable {
+    String path = "hdfs:///tmp/" + dbName + "/";
+    String exportMDPath = "'" + path + "1/'";
+    String exportDataPath = "'" + path + "2/'";
+    String exportDataPathRepl = "'" + path + "3/'";
+    dumpExternalWarehouse.run("create external table " + dbName + ".t1 (i int)")
+            .run("insert into table " + dbName + ".t1 values (1),(2)")
+            .run("export table " + dbName + ".t1 to " + exportDataPathRepl + " for replication('2')")
+            .run("export table " + dbName + ".t1 to " + exportMDPath + " for metadata replication('1')")
+            .run("export table " + dbName + ".t1 to " + exportDataPath);
+
+    destHiveWarehouse.run("use " + replDbName)
+            .run("import table " + replDbName + ".t1 from " + exportMDPath)
+            .run("show tables like 't1'")
+            .verifyResult("t1")
+            .run("import table " + replDbName + ".t2 from " + exportDataPath)
+            .run("select * from " + replDbName + ".t2")
+            .verifyResults(new String[] {"1", "2" })
+            .run("import table " + replDbName + ".t3 from " + exportDataPathRepl)
+            .run("select * from " + replDbName + ".t3")
+            .verifyResults(new String[] {"1", "2" });
+  }
+
+  @Test
   public void databaseTheTableIsImportedIntoShouldBeParsedFromCommandLine() throws Throwable {
     String path = "hdfs:///tmp/" + dbName + "/";
     String exportPath = "'" + path + "1/'";

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcrossInstances.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcrossInstances.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcrossInstances.java
index 08f0130..ff7f9bc 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcrossInstances.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenariosAcrossInstances.java
@@ -1313,4 +1313,81 @@ public class TestReplicationScenariosAcrossInstances {
             .run("show functions like '" + replicatedDbName + "*'")
             .verifyResult(replicatedDbName + ".testFunctionOne");
   }
+
+  @Test
+  public void testDumpExternalTableSetFalse() throws Throwable {
+    WarehouseInstance.Tuple tuple = primary
+            .run("use " + primaryDbName)
+            .run("create external table t1 (id int)")
+            .run("insert into table t1 values (1)")
+            .run("insert into table t1 values (2)")
+            .run("create external table t2 (place string) partitioned by (country string)")
+            .run("insert into table t2 partition(country='india') values ('bangalore')")
+            .run("insert into table t2 partition(country='us') values ('austin')")
+            .run("insert into table t2 partition(country='france') values ('paris')")
+            .dump(primaryDbName, null);
+
+    replica.load(replicatedDbName, tuple.dumpLocation)
+            .run("repl status " + replicatedDbName)
+            .verifyResult(tuple.lastReplicationId)
+            .run("use " + replicatedDbName)
+            .run("show tables like 't1'")
+            .verifyFailure(new String[] {"t1"})
+            .run("show tables like 't2'")
+            .verifyFailure(new String[] {"t2"});
+
+    tuple = primary.run("use " + primaryDbName)
+            .run("create external table t3 (id int)")
+            .run("insert into table t3 values (10)")
+            .run("insert into table t3 values (20)")
+            .dump("repl dump " + primaryDbName + " from " + tuple.lastReplicationId
+                    + " with ('hive.repl.dump.metadata.only'='true')");
+
+    replica.load(replicatedDbName, tuple.dumpLocation)
+            .run("use " + replicatedDbName)
+            .run("show tables like 't3'")
+            .verifyResult("t3")
+            .run("select id from t3 where id = 10")
+            .verifyFailure(new String[] {"10"});
+  }
+
+  @Test
+  public void testDumpExternalTableSetTrue() throws Throwable {
+    WarehouseInstance.Tuple tuple = primary
+            .run("use " + primaryDbName)
+            .run("create external table t1 (id int)")
+            .run("insert into table t1 values (1)")
+            .run("insert into table t1 values (2)")
+            .run("create external table t2 (place string) partitioned by (country string)")
+            .run("insert into table t2 partition(country='india') values ('bangalore')")
+            .run("insert into table t2 partition(country='us') values ('austin')")
+            .run("insert into table t2 partition(country='france') values ('paris')")
+            .dump("repl dump " + primaryDbName + " with ('hive.repl.include.external.tables'='true')");
+
+    replica.load(replicatedDbName, tuple.dumpLocation)
+            .run("use " + replicatedDbName)
+            .run("show tables like 't1'")
+            .verifyResult("t1")
+            .run("show tables like 't2'")
+            .verifyResult("t2")
+            .run("repl status " + replicatedDbName)
+            .verifyResult(tuple.lastReplicationId)
+            .run("select country from t2 where country = 'us'")
+            .verifyResult("us")
+            .run("select country from t2 where country = 'france'")
+            .verifyResult("france");
+
+    tuple = primary.run("use " + primaryDbName)
+            .run("create external table t3 (id int)")
+            .run("insert into table t3 values (10)")
+            .dump("repl dump " + primaryDbName + " from " + tuple.lastReplicationId
+                    + " with ('hive.repl.include.external.tables'='true')");
+
+    replica.load(replicatedDbName, tuple.dumpLocation)
+            .run("use " + replicatedDbName)
+            .run("show tables like 't3'")
+            .verifyResult("t3")
+            .run("select id from t3")
+            .verifyResult("10");
+  }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java b/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
index b2c9daa..06d0ed3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ErrorMsg.java
@@ -199,8 +199,8 @@ public enum ErrorMsg {
   NEED_TABLE_SPECIFICATION(10117, "Table name could be determined; It should be specified "),
   PARTITION_EXISTS(10118, "Partition already exists"),
   TABLE_DATA_EXISTS(10119, "Table exists and contains data files"),
-  INCOMPATIBLE_SCHEMA(10120, "The existing table is not compatible with the import spec. "),
-  EXIM_FOR_NON_NATIVE(10121, "Export/Import cannot be done for a non-native table. "),
+  INCOMPATIBLE_SCHEMA(10120, "The existing table is not compatible with the Export/Import spec. "),
+  EXIM_FOR_NON_NATIVE(10121, "Export/Import cannot be done for a non-native table."),
   INSERT_INTO_BUCKETIZED_TABLE(10122, "Bucketized tables do not support INSERT INTO:"),
   PARTSPEC_DIFFER_FROM_SCHEMA(10125, "Partition columns in partition specification are "
       + "not the same as that defined in the table schema. "

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/java/org/apache/hadoop/hive/ql/exec/ExportTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExportTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExportTask.java
index 3c6a606..078691c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ExportTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ExportTask.java
@@ -55,7 +55,7 @@ public class ExportTask extends Task<ExportWork> implements Serializable {
       TableExport tableExport = new TableExport(exportPaths, work.getTableSpec(),
           work.getReplicationSpec(), db, null, conf, work.getMmContext());
       if (!tableExport.write()) {
-        throw new SemanticException(ErrorMsg.EXIM_FOR_NON_NATIVE.getMsg());
+        throw new SemanticException(ErrorMsg.INCOMPATIBLE_SCHEMA.getMsg());
       }
     } catch (Exception e) {
       LOG.error("failed", e);

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/Utils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/Utils.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/Utils.java
index 976104c..c0701c5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/Utils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/Utils.java
@@ -23,6 +23,7 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.NotificationEvent;
 import org.apache.hadoop.hive.ql.ErrorMsg;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.metadata.Hive;
@@ -180,6 +181,11 @@ public class Utils {
     }
 
     if (replicationSpec.isInReplicationScope()) {
+      if (!hiveConf.getBoolVar(HiveConf.ConfVars.REPL_INCLUDE_EXTERNAL_TABLES) &&
+              MetaStoreUtils.isExternalTable(tableHandle.getTTable()) && !replicationSpec.isMetadataOnly()) {
+        return false;
+      }
+
       boolean isAcidTable = AcidUtils.isTransactionalTable(tableHandle);
       if (isAcidTable) {
         return hiveConf.getBoolVar(HiveConf.ConfVars.REPL_DUMP_INCLUDE_ACID_TABLES);

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/test/queries/clientpositive/repl_2_exim_basic.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/repl_2_exim_basic.q b/ql/src/test/queries/clientpositive/repl_2_exim_basic.q
index 0a2a53d..5b75ca8 100644
--- a/ql/src/test/queries/clientpositive/repl_2_exim_basic.q
+++ b/ql/src/test/queries/clientpositive/repl_2_exim_basic.q
@@ -2,6 +2,7 @@ set hive.mapred.mode=nonstrict;
 set hive.test.mode=true;
 set hive.test.mode.prefix=;
 set hive.test.mode.nosamplelist=managed_t,ext_t,managed_t_imported,managed_t_r_imported,ext_t_imported,ext_t_r_imported;
+set hive.repl.include.external.tables=true;
 
 drop table if exists managed_t;
 drop table if exists ext_t;

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/test/results/clientnegative/exim_03_nonpart_noncompat_colschema.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/exim_03_nonpart_noncompat_colschema.q.out b/ql/src/test/results/clientnegative/exim_03_nonpart_noncompat_colschema.q.out
index b61c38c..10ce3e9 100644
--- a/ql/src/test/results/clientnegative/exim_03_nonpart_noncompat_colschema.q.out
+++ b/ql/src/test/results/clientnegative/exim_03_nonpart_noncompat_colschema.q.out
@@ -59,4 +59,4 @@ POSTHOOK: query: create table exim_department ( dep_key int comment "department
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:importer
 POSTHOOK: Output: importer@exim_department
-FAILED: SemanticException [Error 10120]: The existing table is not compatible with the import spec.   Column Schema does not match
+FAILED: SemanticException [Error 10120]: The existing table is not compatible with the Export/Import spec.   Column Schema does not match

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/test/results/clientnegative/exim_04_nonpart_noncompat_colnumber.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/exim_04_nonpart_noncompat_colnumber.q.out b/ql/src/test/results/clientnegative/exim_04_nonpart_noncompat_colnumber.q.out
index f8f5ea7..c53407b 100644
--- a/ql/src/test/results/clientnegative/exim_04_nonpart_noncompat_colnumber.q.out
+++ b/ql/src/test/results/clientnegative/exim_04_nonpart_noncompat_colnumber.q.out
@@ -59,4 +59,4 @@ POSTHOOK: query: create table exim_department ( dep_id int comment "department i
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:importer
 POSTHOOK: Output: importer@exim_department
-FAILED: SemanticException [Error 10120]: The existing table is not compatible with the import spec.   Column Schema does not match
+FAILED: SemanticException [Error 10120]: The existing table is not compatible with the Export/Import spec.   Column Schema does not match

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/test/results/clientnegative/exim_05_nonpart_noncompat_coltype.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/exim_05_nonpart_noncompat_coltype.q.out b/ql/src/test/results/clientnegative/exim_05_nonpart_noncompat_coltype.q.out
index f118aa7..4806861 100644
--- a/ql/src/test/results/clientnegative/exim_05_nonpart_noncompat_coltype.q.out
+++ b/ql/src/test/results/clientnegative/exim_05_nonpart_noncompat_coltype.q.out
@@ -59,4 +59,4 @@ POSTHOOK: query: create table exim_department ( dep_id bigint comment "departmen
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:importer
 POSTHOOK: Output: importer@exim_department
-FAILED: SemanticException [Error 10120]: The existing table is not compatible with the import spec.   Column Schema does not match
+FAILED: SemanticException [Error 10120]: The existing table is not compatible with the Export/Import spec.   Column Schema does not match

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/test/results/clientnegative/exim_06_nonpart_noncompat_storage.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/exim_06_nonpart_noncompat_storage.q.out b/ql/src/test/results/clientnegative/exim_06_nonpart_noncompat_storage.q.out
index 4b0fbdb..748d596 100644
--- a/ql/src/test/results/clientnegative/exim_06_nonpart_noncompat_storage.q.out
+++ b/ql/src/test/results/clientnegative/exim_06_nonpart_noncompat_storage.q.out
@@ -59,4 +59,4 @@ POSTHOOK: query: create table exim_department ( dep_id int comment "department i
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:importer
 POSTHOOK: Output: importer@exim_department
-FAILED: SemanticException [Error 10120]: The existing table is not compatible with the import spec.   Table inputformat/outputformats do not match
+FAILED: SemanticException [Error 10120]: The existing table is not compatible with the Export/Import spec.   Table inputformat/outputformats do not match

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/test/results/clientnegative/exim_07_nonpart_noncompat_ifof.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/exim_07_nonpart_noncompat_ifof.q.out b/ql/src/test/results/clientnegative/exim_07_nonpart_noncompat_ifof.q.out
index 2c63d1b..7c3ba02 100644
--- a/ql/src/test/results/clientnegative/exim_07_nonpart_noncompat_ifof.q.out
+++ b/ql/src/test/results/clientnegative/exim_07_nonpart_noncompat_ifof.q.out
@@ -65,4 +65,4 @@ POSTHOOK: query: create table exim_department ( dep_id int comment "department i
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:importer
 POSTHOOK: Output: importer@exim_department
-FAILED: SemanticException [Error 10120]: The existing table is not compatible with the import spec.   Table inputformat/outputformats do not match
+FAILED: SemanticException [Error 10120]: The existing table is not compatible with the Export/Import spec.   Table inputformat/outputformats do not match

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/test/results/clientnegative/exim_08_nonpart_noncompat_serde.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/exim_08_nonpart_noncompat_serde.q.out b/ql/src/test/results/clientnegative/exim_08_nonpart_noncompat_serde.q.out
index 6098ad5..eb3715f 100644
--- a/ql/src/test/results/clientnegative/exim_08_nonpart_noncompat_serde.q.out
+++ b/ql/src/test/results/clientnegative/exim_08_nonpart_noncompat_serde.q.out
@@ -61,4 +61,4 @@ POSTHOOK: query: create table exim_department ( dep_id int comment "department i
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:importer
 POSTHOOK: Output: importer@exim_department
-FAILED: SemanticException [Error 10120]: The existing table is not compatible with the import spec.   Table Serde class does not match
+FAILED: SemanticException [Error 10120]: The existing table is not compatible with the Export/Import spec.   Table Serde class does not match

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/test/results/clientnegative/exim_09_nonpart_noncompat_serdeparam.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/exim_09_nonpart_noncompat_serdeparam.q.out b/ql/src/test/results/clientnegative/exim_09_nonpart_noncompat_serdeparam.q.out
index e3cc024..d076829 100644
--- a/ql/src/test/results/clientnegative/exim_09_nonpart_noncompat_serdeparam.q.out
+++ b/ql/src/test/results/clientnegative/exim_09_nonpart_noncompat_serdeparam.q.out
@@ -69,4 +69,4 @@ POSTHOOK: query: create table exim_department ( dep_id int comment "department i
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:importer
 POSTHOOK: Output: importer@exim_department
-FAILED: SemanticException [Error 10120]: The existing table is not compatible with the import spec.   Table Serde format does not match
+FAILED: SemanticException [Error 10120]: The existing table is not compatible with the Export/Import spec.   Table Serde format does not match

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/test/results/clientnegative/exim_10_nonpart_noncompat_bucketing.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/exim_10_nonpart_noncompat_bucketing.q.out b/ql/src/test/results/clientnegative/exim_10_nonpart_noncompat_bucketing.q.out
index 8dc3df0..0ad8410 100644
--- a/ql/src/test/results/clientnegative/exim_10_nonpart_noncompat_bucketing.q.out
+++ b/ql/src/test/results/clientnegative/exim_10_nonpart_noncompat_bucketing.q.out
@@ -61,4 +61,4 @@ POSTHOOK: query: create table exim_department ( dep_id int comment "department i
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:importer
 POSTHOOK: Output: importer@exim_department
-FAILED: SemanticException [Error 10120]: The existing table is not compatible with the import spec.   Table bucketing spec does not match
+FAILED: SemanticException [Error 10120]: The existing table is not compatible with the Export/Import spec.   Table bucketing spec does not match

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/test/results/clientnegative/exim_11_nonpart_noncompat_sorting.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/exim_11_nonpart_noncompat_sorting.q.out b/ql/src/test/results/clientnegative/exim_11_nonpart_noncompat_sorting.q.out
index 872d5e6..abd0d82 100644
--- a/ql/src/test/results/clientnegative/exim_11_nonpart_noncompat_sorting.q.out
+++ b/ql/src/test/results/clientnegative/exim_11_nonpart_noncompat_sorting.q.out
@@ -63,4 +63,4 @@ POSTHOOK: query: create table exim_department ( dep_id int comment "department i
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:importer
 POSTHOOK: Output: importer@exim_department
-FAILED: SemanticException [Error 10120]: The existing table is not compatible with the import spec.   Table sorting spec does not match
+FAILED: SemanticException [Error 10120]: The existing table is not compatible with the Export/Import spec.   Table sorting spec does not match

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/test/results/clientnegative/exim_12_nonnative_export.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/exim_12_nonnative_export.q.out b/ql/src/test/results/clientnegative/exim_12_nonnative_export.q.out
index bd73536..4546490 100644
--- a/ql/src/test/results/clientnegative/exim_12_nonnative_export.q.out
+++ b/ql/src/test/results/clientnegative/exim_12_nonnative_export.q.out
@@ -16,4 +16,4 @@ PREHOOK: query: export table exim_department to 'ql/test/data/exports/exim_depar
 PREHOOK: type: EXPORT
 PREHOOK: Input: default@exim_department
 #### A masked pattern was here ####
-FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.ExportTask. Export/Import cannot be done for a non-native table. 
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.ExportTask. The existing table is not compatible with the Export/Import spec. 

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/test/results/clientnegative/exim_13_nonnative_import.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/exim_13_nonnative_import.q.out b/ql/src/test/results/clientnegative/exim_13_nonnative_import.q.out
index d820643..ab45089 100644
--- a/ql/src/test/results/clientnegative/exim_13_nonnative_import.q.out
+++ b/ql/src/test/results/clientnegative/exim_13_nonnative_import.q.out
@@ -59,4 +59,4 @@ POSTHOOK: query: create table exim_department ( dep_id int comment "department i
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:importer
 POSTHOOK: Output: importer@exim_department
-FAILED: SemanticException [Error 10121]: Export/Import cannot be done for a non-native table. 
+FAILED: SemanticException [Error 10121]: Export/Import cannot be done for a non-native table.

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/test/results/clientnegative/exim_14_nonpart_part.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/exim_14_nonpart_part.q.out b/ql/src/test/results/clientnegative/exim_14_nonpart_part.q.out
index da116ef..f5d19a8 100644
--- a/ql/src/test/results/clientnegative/exim_14_nonpart_part.q.out
+++ b/ql/src/test/results/clientnegative/exim_14_nonpart_part.q.out
@@ -61,4 +61,4 @@ POSTHOOK: query: create table exim_department ( dep_id int comment "department i
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:importer
 POSTHOOK: Output: importer@exim_department
-FAILED: SemanticException [Error 10120]: The existing table is not compatible with the import spec.   Partition Schema does not match
+FAILED: SemanticException [Error 10120]: The existing table is not compatible with the Export/Import spec.   Partition Schema does not match

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/test/results/clientnegative/exim_15_part_nonpart.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/exim_15_part_nonpart.q.out b/ql/src/test/results/clientnegative/exim_15_part_nonpart.q.out
index dfbf025..3bb0fce 100644
--- a/ql/src/test/results/clientnegative/exim_15_part_nonpart.q.out
+++ b/ql/src/test/results/clientnegative/exim_15_part_nonpart.q.out
@@ -62,4 +62,4 @@ POSTHOOK: query: create table exim_department ( dep_id int comment "department i
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:importer
 POSTHOOK: Output: importer@exim_department
-FAILED: SemanticException [Error 10120]: The existing table is not compatible with the import spec.   Partition Schema does not match
+FAILED: SemanticException [Error 10120]: The existing table is not compatible with the Export/Import spec.   Partition Schema does not match

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/test/results/clientnegative/exim_16_part_noncompat_schema.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/exim_16_part_noncompat_schema.q.out b/ql/src/test/results/clientnegative/exim_16_part_noncompat_schema.q.out
index 4cb6ca7..302e511 100644
--- a/ql/src/test/results/clientnegative/exim_16_part_noncompat_schema.q.out
+++ b/ql/src/test/results/clientnegative/exim_16_part_noncompat_schema.q.out
@@ -64,4 +64,4 @@ POSTHOOK: query: create table exim_department ( dep_id int comment "department i
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:importer
 POSTHOOK: Output: importer@exim_department
-FAILED: SemanticException [Error 10120]: The existing table is not compatible with the import spec.   Partition Schema does not match
+FAILED: SemanticException [Error 10120]: The existing table is not compatible with the Export/Import spec.   Partition Schema does not match

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/test/results/clientnegative/exim_19_external_over_existing.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/exim_19_external_over_existing.q.out b/ql/src/test/results/clientnegative/exim_19_external_over_existing.q.out
index 763efb2..db01d4f 100644
--- a/ql/src/test/results/clientnegative/exim_19_external_over_existing.q.out
+++ b/ql/src/test/results/clientnegative/exim_19_external_over_existing.q.out
@@ -59,4 +59,4 @@ POSTHOOK: query: create  table exim_department ( dep_id int comment "department
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:importer
 POSTHOOK: Output: importer@exim_department
-FAILED: SemanticException [Error 10120]: The existing table is not compatible with the import spec.   External table cannot overwrite existing table. Drop existing table first.
+FAILED: SemanticException [Error 10120]: The existing table is not compatible with the Export/Import spec.   External table cannot overwrite existing table. Drop existing table first.

http://git-wip-us.apache.org/repos/asf/hive/blob/9bc90f28/ql/src/test/results/clientnegative/exim_21_part_managed_external.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientnegative/exim_21_part_managed_external.q.out b/ql/src/test/results/clientnegative/exim_21_part_managed_external.q.out
index fd27f29..ebcc8db 100644
--- a/ql/src/test/results/clientnegative/exim_21_part_managed_external.q.out
+++ b/ql/src/test/results/clientnegative/exim_21_part_managed_external.q.out
@@ -109,4 +109,4 @@ POSTHOOK: query: create table exim_employee ( emp_id int comment "employee id")
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:importer
 POSTHOOK: Output: importer@exim_employee
-FAILED: SemanticException [Error 10120]: The existing table is not compatible with the import spec.   External table cannot overwrite existing table. Drop existing table first.
+FAILED: SemanticException [Error 10120]: The existing table is not compatible with the Export/Import spec.   External table cannot overwrite existing table. Drop existing table first.