diff --git a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseMetaHook.java b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseMetaHook.java index 3565c8c813..f01ed5760a 100644 --- a/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseMetaHook.java +++ b/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseMetaHook.java @@ -32,6 +32,8 @@ import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils; import org.apache.hadoop.util.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.Closeable; import java.io.IOException; @@ -44,6 +46,7 @@ * be used after usage. */ public class HBaseMetaHook implements HiveMetaHook, Closeable { + private static final Logger LOG = LoggerFactory.getLogger(HBaseMetaHook.class); private Configuration hbaseConf; private Admin admin; @@ -99,12 +102,15 @@ public void rollbackDropTable(Table table) throws MetaException { public void commitDropTable(Table tbl, boolean deleteData) throws MetaException { try { String tableName = getHBaseTableName(tbl); - boolean isExternal = MetaStoreUtils.isExternalTable(tbl); - if (deleteData && !isExternal) { - if (getHBaseAdmin().isTableEnabled(TableName.valueOf(tableName))) { - getHBaseAdmin().disableTable(TableName.valueOf(tableName)); + boolean isPurge = !MetaStoreUtils.isExternalTable(tbl) || MetaStoreUtils.isExternalTablePurge(tbl); + if (deleteData && isPurge) { + LOG.info("Dropping with purge all the data for data source {}", tableName); + if (getHBaseAdmin().tableExists(TableName.valueOf(tableName))) { + if (getHBaseAdmin().isTableEnabled(TableName.valueOf(tableName))) { + getHBaseAdmin().disableTable(TableName.valueOf(tableName)); + } + getHBaseAdmin().deleteTable(TableName.valueOf(tableName)); } - getHBaseAdmin().deleteTable(TableName.valueOf(tableName)); } } catch (IOException ie) { throw new MetaException(StringUtils.stringifyException(ie)); @@ -113,8 +119,6 @@ public void commitDropTable(Table tbl, boolean deleteData) throws MetaException @Override public void preCreateTable(Table tbl) throws MetaException { - boolean isExternal = MetaStoreUtils.isExternalTable(tbl); - // We'd like to move this to HiveMetaStore for any non-native table, but // first we need to support storing NULL for location on a table if (tbl.getSd().getLocation() != null) { @@ -133,34 +137,24 @@ public void preCreateTable(Table tbl) throws MetaException { HTableDescriptor tableDesc; if (!getHBaseAdmin().tableExists(TableName.valueOf(tableName))) { - // if it is not an external table then create one - if (!isExternal) { - // Create the column descriptors - tableDesc = new HTableDescriptor(TableName.valueOf(tableName)); - Set uniqueColumnFamilies = new HashSet(); - - for (ColumnMappings.ColumnMapping colMap : columnMappings) { - if (!colMap.hbaseRowKey && !colMap.hbaseTimestamp) { - uniqueColumnFamilies.add(colMap.familyName); - } - } + // create table from Hive + // create the column descriptors + tableDesc = new HTableDescriptor(TableName.valueOf(tableName)); + Set uniqueColumnFamilies = new HashSet(); - for (String columnFamily : uniqueColumnFamilies) { - tableDesc.addFamily(new HColumnDescriptor(Bytes.toBytes(columnFamily))); + for (ColumnMappings.ColumnMapping colMap : columnMappings) { + if (!colMap.hbaseRowKey && !colMap.hbaseTimestamp) { + uniqueColumnFamilies.add(colMap.familyName); } + } - getHBaseAdmin().createTable(tableDesc); - } else { - // an external table - throw new MetaException("HBase table " + tableName + - " doesn't exist while the table is declared as an external table."); + for (String columnFamily : uniqueColumnFamilies) { + tableDesc.addFamily(new HColumnDescriptor(Bytes.toBytes(columnFamily))); } + getHBaseAdmin().createTable(tableDesc); } else { - if (!isExternal) { - throw new MetaException("Table " + tableName + " already exists within HBase; " - + "use CREATE EXTERNAL TABLE instead to register it in Hive."); - } + // register table in Hive // make sure the schema mapping is right tableDesc = getHBaseAdmin().getTableDescriptor(TableName.valueOf(tableName)); @@ -190,10 +184,10 @@ public void preCreateTable(Table tbl) throws MetaException { @Override public void rollbackCreateTable(Table table) throws MetaException { - boolean isExternal = MetaStoreUtils.isExternalTable(table); String tableName = getHBaseTableName(table); + boolean isPurge = !MetaStoreUtils.isExternalTable(table) || MetaStoreUtils.isExternalTablePurge(table); try { - if (!isExternal && getHBaseAdmin().tableExists(TableName.valueOf(tableName))) { + if (isPurge && getHBaseAdmin().tableExists(TableName.valueOf(tableName))) { // we have created an HBase table, so we delete it to roll back; if (getHBaseAdmin().isTableEnabled(TableName.valueOf(tableName))) { getHBaseAdmin().disableTable(TableName.valueOf(tableName)); diff --git a/hbase-handler/src/test/queries/negative/cascade_dbdrop.q b/hbase-handler/src/test/queries/negative/cascade_dbdrop.q index 7f9df5eb2f..99e36ac7a6 100644 --- a/hbase-handler/src/test/queries/negative/cascade_dbdrop.q +++ b/hbase-handler/src/test/queries/negative/cascade_dbdrop.q @@ -11,10 +11,10 @@ CREATE DATABASE hbaseDB; -- Exit Code < 0 on syntax/usage error -- Exit Code > 0 operation failed -CREATE TABLE hbaseDB.hbase_table_0(key int, value string) +CREATE EXTERNAL TABLE hbaseDB.hbase_table_0(key int, value string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,cf:string") -TBLPROPERTIES ("hbase.table.name" = "hbase_table_0"); +TBLPROPERTIES ("hbase.table.name" = "hbase_table_0", "external.table.purge" = "true"); dfs -ls target/tmp/hbase/data/default/hbase_table_0; diff --git a/hbase-handler/src/test/queries/negative/cascade_dbdrop_hadoop20.q b/hbase-handler/src/test/queries/negative/cascade_dbdrop_hadoop20.q index 8fa8c8acbb..b06314a469 100644 --- a/hbase-handler/src/test/queries/negative/cascade_dbdrop_hadoop20.q +++ b/hbase-handler/src/test/queries/negative/cascade_dbdrop_hadoop20.q @@ -11,19 +11,13 @@ CREATE DATABASE hbaseDB; -- Exit Code < 0 on syntax/usage error -- Exit Code > 0 operation failed -CREATE TABLE hbaseDB.hbase_table_0(key int, value string) +CREATE EXTERNAL TABLE hbaseDB.hbase_table_0(key int, value string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,cf:string") -TBLPROPERTIES ("hbase.table.name" = "hbase_table_0"); +TBLPROPERTIES ("hbase.table.name" = "hbase_table_0", "external.table.purge" = "true"); dfs -ls target/tmp/hbase/data/default/hbase_table_0; DROP DATABASE IF EXISTS hbaseDB CASCADE; dfs -ls target/tmp/hbase/data/hbase/default/hbase_table_0; - - - - - - diff --git a/hbase-handler/src/test/queries/negative/generatehfiles_require_family_path.q b/hbase-handler/src/test/queries/negative/generatehfiles_require_family_path.q index 6844fbc94c..ceb9390623 100644 --- a/hbase-handler/src/test/queries/negative/generatehfiles_require_family_path.q +++ b/hbase-handler/src/test/queries/negative/generatehfiles_require_family_path.q @@ -2,9 +2,10 @@ DROP TABLE IF EXISTS hbase_bulk; -CREATE TABLE hbase_bulk (key INT, value STRING) +CREATE EXTERNAL TABLE hbase_bulk (key INT, value STRING) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' -WITH SERDEPROPERTIES ('hbase.columns.mapping' = ':key,cf:string'); +WITH SERDEPROPERTIES ('hbase.columns.mapping' = ':key,cf:string') +TBLPROPERTIES ("external.table.purge" = "true"); SET hive.hbase.generatehfiles = true; INSERT OVERWRITE TABLE hbase_bulk SELECT * FROM src CLUSTER BY key; diff --git a/hbase-handler/src/test/queries/negative/hbase_ddl.q b/hbase-handler/src/test/queries/negative/hbase_ddl.q index 2913bcdabd..24f684c267 100644 --- a/hbase-handler/src/test/queries/negative/hbase_ddl.q +++ b/hbase-handler/src/test/queries/negative/hbase_ddl.q @@ -1,8 +1,8 @@ DROP TABLE hbase_table_1; -CREATE TABLE hbase_table_1(key int comment 'It is a column key', value string comment 'It is the column string value') +CREATE EXTERNAL TABLE hbase_table_1(key int comment 'It is a column key', value string comment 'It is the column string value') STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = "cf:string") -TBLPROPERTIES ("hbase.table.name" = "hbase_table_0"); +TBLPROPERTIES ("hbase.table.name" = "hbase_table_0", "external.table.purge" = "true"); DESCRIBE EXTENDED hbase_table_1; diff --git a/hbase-handler/src/test/queries/positive/external_table_ppd.q b/hbase-handler/src/test/queries/positive/external_table_ppd.q index fbef4bb578..3c10cd1ba6 100644 --- a/hbase-handler/src/test/queries/positive/external_table_ppd.q +++ b/hbase-handler/src/test/queries/positive/external_table_ppd.q @@ -1,6 +1,6 @@ DROP TABLE t_hbase; -CREATE TABLE t_hbase(key STRING, +CREATE EXTERNAL TABLE t_hbase(key STRING, tinyint_col TINYINT, smallint_col SMALLINT, int_col INT, @@ -11,7 +11,8 @@ CREATE TABLE t_hbase(key STRING, STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = "cf:binarykey#-,cf:binarybyte#-,cf:binaryshort#-,:key#-,cf:binarylong#-,cf:binaryfloat#-,cf:binarydouble#-,cf:binaryboolean#-") TBLPROPERTIES ("hbase.table.name" = "t_hive", - "hbase.table.default.storage.type" = "binary"); + "hbase.table.default.storage.type" = "binary", + "external.table.purge" = "true"); DESCRIBE FORMATTED t_hbase; diff --git a/hbase-handler/src/test/queries/positive/hbase_binary_binary.q b/hbase-handler/src/test/queries/positive/hbase_binary_binary.q index 388e5aad45..7c17ca2d23 100644 --- a/hbase-handler/src/test/queries/positive/hbase_binary_binary.q +++ b/hbase-handler/src/test/queries/positive/hbase_binary_binary.q @@ -1,9 +1,10 @@ drop table if exists testhbaseb; -CREATE TABLE testhbaseb (key int, val binary) +CREATE EXTERNAL TABLE testhbaseb (key int, val binary) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.columns.mapping" = ":key,cf:val#b" -); +) +TBLPROPERTIES ("external.table.purge" = "true"); insert into table testhbaseb values(1, 'hello'); insert into table testhbaseb values(2, 'hi'); select * from testhbaseb; diff --git a/hbase-handler/src/test/queries/positive/hbase_binary_map_queries.q b/hbase-handler/src/test/queries/positive/hbase_binary_map_queries.q index 255a2c74d2..af3ee4f48c 100644 --- a/hbase-handler/src/test/queries/positive/hbase_binary_map_queries.q +++ b/hbase-handler/src/test/queries/positive/hbase_binary_map_queries.q @@ -1,13 +1,14 @@ DROP TABLE hbase_src; -CREATE TABLE hbase_src(key STRING, +CREATE EXTERNAL TABLE hbase_src(key STRING, tinyint_col TINYINT, smallint_col SMALLINT, int_col INT, bigint_col BIGINT, float_col FLOAT, double_col DOUBLE, - string_col STRING); + string_col STRING) +TBLPROPERTIES ("external.table.purge" = "true"); INSERT OVERWRITE TABLE hbase_src SELECT key, key, key, key, key, key, key, value @@ -16,7 +17,7 @@ INSERT OVERWRITE TABLE hbase_src DROP TABLE t_hbase_maps; -CREATE TABLE t_hbase_maps(key STRING, +CREATE EXTERNAL TABLE t_hbase_maps(key STRING, tinyint_map_col MAP, smallint_map_col MAP, int_map_col MAP, @@ -27,7 +28,7 @@ CREATE TABLE t_hbase_maps(key STRING, boolean_map_col MAP) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping"=":key,cf-tinyint:,cf-smallint:,cf-int:,cf-bigint:,cf-float:,cf-double:,cf-string:,cf-boolean:") -TBLPROPERTIES ("hbase.table.name"="t_hive_maps"); +TBLPROPERTIES ("hbase.table.name"="t_hive_maps", "external.table.purge" = "true"); INSERT OVERWRITE TABLE t_hbase_maps SELECT key, @@ -118,7 +119,7 @@ DROP TABLE t_ext_hbase_maps_2; DROP TABLE t_hbase_maps_1; -CREATE TABLE t_hbase_maps_1(key STRING, +CREATE EXTERNAL TABLE t_hbase_maps_1(key STRING, tinyint_map_col MAP, smallint_map_col MAP, int_map_col MAP, @@ -129,7 +130,7 @@ CREATE TABLE t_hbase_maps_1(key STRING, boolean_map_col MAP) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping"=":key#b,cf-tinyint:#b:b,cf-smallint:#b:b,cf-int:#b:b,cf-bigint:#b:b,cf-float:#b:b,cf-double:#b:b,cf-string:#b:b,cf-boolean:#b:b") -TBLPROPERTIES ("hbase.table.name"="t_hive_maps_1"); +TBLPROPERTIES ("hbase.table.name"="t_hive_maps_1", "external.table.purge" = "true"); INSERT OVERWRITE TABLE t_hbase_maps_1 SELECT key, diff --git a/hbase-handler/src/test/queries/positive/hbase_binary_map_queries_prefix.q b/hbase-handler/src/test/queries/positive/hbase_binary_map_queries_prefix.q index 9ff4366753..ead792beb3 100644 --- a/hbase-handler/src/test/queries/positive/hbase_binary_map_queries_prefix.q +++ b/hbase-handler/src/test/queries/positive/hbase_binary_map_queries_prefix.q @@ -1,13 +1,14 @@ DROP TABLE hbase_src; -CREATE TABLE hbase_src(key STRING, +CREATE EXTERNAL TABLE hbase_src(key STRING, tinyint_col TINYINT, smallint_col SMALLINT, int_col INT, bigint_col BIGINT, float_col FLOAT, double_col DOUBLE, - string_col STRING); + string_col STRING) +TBLPROPERTIES ("external.table.purge" = "true"); INSERT OVERWRITE TABLE hbase_src SELECT key, key, key, key, key, key, key, value @@ -16,12 +17,12 @@ INSERT OVERWRITE TABLE hbase_src DROP TABLE t_hbase_maps; -CREATE TABLE t_hbase_maps(key STRING, +CREATE EXTERNAL TABLE t_hbase_maps(key STRING, string_map_col MAP, simple_string_col STRING) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping"=":key,cf-string:,cf-string:simple_string_col") -TBLPROPERTIES ("hbase.table.name"="t_hive_maps"); +TBLPROPERTIES ("hbase.table.name"="t_hive_maps", "external.table.purge" = "true"); INSERT OVERWRITE TABLE t_hbase_maps SELECT key, diff --git a/hbase-handler/src/test/queries/positive/hbase_binary_storage_queries.q b/hbase-handler/src/test/queries/positive/hbase_binary_storage_queries.q index b048871190..4e62e0f9b5 100644 --- a/hbase-handler/src/test/queries/positive/hbase_binary_storage_queries.q +++ b/hbase-handler/src/test/queries/positive/hbase_binary_storage_queries.q @@ -1,6 +1,6 @@ DROP TABLE t_hbase; -CREATE TABLE t_hbase(key STRING, +CREATE EXTERNAL TABLE t_hbase(key STRING, tinyint_col TINYINT, smallint_col SMALLINT, int_col INT, @@ -11,7 +11,8 @@ CREATE TABLE t_hbase(key STRING, STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key#-,cf:binarybyte#-,cf:binaryshort#-,cf:binaryint#-,cf:binarylong#-,cf:binaryfloat#-,cf:binarydouble#-,cf:binaryboolean#-") TBLPROPERTIES ("hbase.table.name" = "t_hive", - "hbase.table.default.storage.type" = "binary"); + "hbase.table.default.storage.type" = "binary", + "external.table.purge" = "true"); DESCRIBE FORMATTED t_hbase; @@ -92,7 +93,7 @@ DROP TABLE t_hbase_1; DROP TABLE t_hbase; DROP TABLE t_hbase_2; -CREATE TABLE t_hbase_2(key STRING, +CREATE EXTERNAL TABLE t_hbase_2(key STRING, tinyint_col TINYINT, smallint_col SMALLINT, int_col INT, @@ -102,7 +103,7 @@ CREATE TABLE t_hbase_2(key STRING, boolean_col BOOLEAN) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key#-,cf:binarybyte#-,cf:binaryshort#-,cf:binaryint#-,cf:binarylong#-,cf:binaryfloat#-,cf:binarydouble#-,cf:binaryboolean#-") -TBLPROPERTIES ("hbase.table.name" = "t_hive_2"); +TBLPROPERTIES ("hbase.table.name" = "t_hive_2", "external.table.purge" = "true"); INSERT OVERWRITE TABLE t_hbase_2 SELECT 'user1', 1, 1, 1, 1, 1.0, 1.0, true @@ -191,7 +192,8 @@ STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key#-,cf:binarybyte#-,cf:binaryshort#-,cf:binaryint#-,cf:binarylong#-,cf:binaryfloat#-,cf:binarydouble#-,cf:binaryboolean#-") TBLPROPERTIES ( "hbase.table.name" = "t_hive_2", -"hbase.table.default.storage.type" = "binary"); +"hbase.table.default.storage.type" = "binary", +"external.table.purge" = "true"); SELECT * FROM t_hbase_4; diff --git a/hbase-handler/src/test/queries/positive/hbase_custom_key.q b/hbase-handler/src/test/queries/positive/hbase_custom_key.q index 9dbb2a0bf6..74560275d2 100644 --- a/hbase-handler/src/test/queries/positive/hbase_custom_key.q +++ b/hbase-handler/src/test/queries/positive/hbase_custom_key.q @@ -1,9 +1,10 @@ -CREATE TABLE hbase_ck_1(key struct, value string) +CREATE EXTERNAL TABLE hbase_ck_1(key struct, value string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.table.name" = "hbase_custom", "hbase.columns.mapping" = ":key,cf:string", - "hbase.composite.key.factory"="org.apache.hadoop.hive.hbase.SampleHBaseKeyFactory"); + "hbase.composite.key.factory"="org.apache.hadoop.hive.hbase.SampleHBaseKeyFactory") +TBLPROPERTIES ("external.table.purge" = "true"); CREATE EXTERNAL TABLE hbase_ck_2(key string, value string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' diff --git a/hbase-handler/src/test/queries/positive/hbase_custom_key2.q b/hbase-handler/src/test/queries/positive/hbase_custom_key2.q index 9fba4f6e78..c846a50722 100644 --- a/hbase-handler/src/test/queries/positive/hbase_custom_key2.q +++ b/hbase-handler/src/test/queries/positive/hbase_custom_key2.q @@ -1,9 +1,10 @@ -CREATE TABLE hbase_ck_4(key struct, value string) +CREATE EXTERNAL TABLE hbase_ck_4(key struct, value string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.table.name" = "hbase_custom2", "hbase.columns.mapping" = ":key,cf:string", - "hbase.composite.key.factory"="org.apache.hadoop.hive.hbase.SampleHBaseKeyFactory2"); + "hbase.composite.key.factory"="org.apache.hadoop.hive.hbase.SampleHBaseKeyFactory2") +TBLPROPERTIES ("external.table.purge" = "true");; from src tablesample (5 rows) insert into table hbase_ck_4 select diff --git a/hbase-handler/src/test/queries/positive/hbase_custom_key3.q b/hbase-handler/src/test/queries/positive/hbase_custom_key3.q index 22d2c9e07d..0bc773a6c7 100644 --- a/hbase-handler/src/test/queries/positive/hbase_custom_key3.q +++ b/hbase-handler/src/test/queries/positive/hbase_custom_key3.q @@ -1,9 +1,10 @@ -CREATE TABLE hbase_ck_5(key struct, value string) +CREATE EXTERNAL TABLE hbase_ck_5(key struct, value string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.table.name" = "hbase_custom3", "hbase.columns.mapping" = ":key,cf:string", - "hbase.composite.key.factory"="org.apache.hadoop.hive.hbase.SampleHBaseKeyFactory3"); + "hbase.composite.key.factory"="org.apache.hadoop.hive.hbase.SampleHBaseKeyFactory3") +TBLPROPERTIES ("external.table.purge" = "true"); from src tablesample (5 rows) insert into table hbase_ck_5 select diff --git a/hbase-handler/src/test/queries/positive/hbase_ddl.q b/hbase-handler/src/test/queries/positive/hbase_ddl.q index a8bae75cec..37f7f617f9 100644 --- a/hbase-handler/src/test/queries/positive/hbase_ddl.q +++ b/hbase-handler/src/test/queries/positive/hbase_ddl.q @@ -1,8 +1,8 @@ DROP TABLE hbase_table_1; -CREATE TABLE hbase_table_1(key int comment 'It is a column key', value string comment 'It is the column string value') +CREATE EXTERNAL TABLE hbase_table_1(key int comment 'It is a column key', value string comment 'It is the column string value') STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = "cf:string") -TBLPROPERTIES ("hbase.table.name" = "hbase_table_0"); +TBLPROPERTIES ("hbase.table.name" = "hbase_table_0", "external.table.purge" = "true"); DESCRIBE EXTENDED hbase_table_1; diff --git a/hbase-handler/src/test/queries/positive/hbase_decimal_decimal.q b/hbase-handler/src/test/queries/positive/hbase_decimal_decimal.q index d943fbd97c..016f10b5cd 100644 --- a/hbase-handler/src/test/queries/positive/hbase_decimal_decimal.q +++ b/hbase-handler/src/test/queries/positive/hbase_decimal_decimal.q @@ -1,11 +1,12 @@ -CREATE TABLE testhbase_decimal ( +CREATE EXTERNAL TABLE testhbase_decimal ( id int, balance decimal(15,2)) ROW FORMAT DELIMITED COLLECTION ITEMS TERMINATED BY '~' STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( -"hbase.columns.mapping"=":key,cf:balance#b"); +"hbase.columns.mapping"=":key,cf:balance#b") +TBLPROPERTIES ("external.table.purge" = "true"); insert into testhbase_decimal values (1,1), (2, 2.2), (3, 33.33); diff --git a/hbase-handler/src/test/queries/positive/hbase_handler_bulk.q b/hbase-handler/src/test/queries/positive/hbase_handler_bulk.q index d02a61f859..159574d68d 100644 --- a/hbase-handler/src/test/queries/positive/hbase_handler_bulk.q +++ b/hbase-handler/src/test/queries/positive/hbase_handler_bulk.q @@ -3,10 +3,10 @@ drop table if exists hb_target; -- this is the target HBase table -create table hb_target(key int, val string) +create external table hb_target(key int, val string) stored by 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' with serdeproperties ('hbase.columns.mapping' = ':key,cf:val') -tblproperties ('hbase.table.name' = 'positive_hbase_handler_bulk'); +tblproperties ('hbase.table.name' = 'positive_hbase_handler_bulk', 'external.table.purge' = 'true'); set hive.hbase.generatehfiles=true; set hfile.family.path=/tmp/hb_target/cf; @@ -27,10 +27,10 @@ drop table hb_target; dfs -rmr /tmp/hb_target/cf; -create table hb_target(key int, val string) +create external table hb_target(key int, val string) stored by 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' with serdeproperties ('hbase.columns.mapping' = ':key,cf:val') -tblproperties ('hbase.table.name' = 'positive_hbase_handler_bulk'); +tblproperties ('hbase.table.name' = 'positive_hbase_handler_bulk', 'external.table.purge' = 'true'); -- do it twice - regression test for HIVE-18607 @@ -42,6 +42,3 @@ insert overwrite table hb_target select distinct key, value from src cluster by drop table hb_target; dfs -rmr /tmp/hb_target/cf; - - - diff --git a/hbase-handler/src/test/queries/positive/hbase_joins.q b/hbase-handler/src/test/queries/positive/hbase_joins.q index 1616adcd08..815e0e33f3 100644 --- a/hbase-handler/src/test/queries/positive/hbase_joins.q +++ b/hbase-handler/src/test/queries/positive/hbase_joins.q @@ -5,23 +5,26 @@ DROP TABLE users_level; -- From HIVE-1257 -CREATE TABLE users(key string, state string, country string, country_id int) +CREATE EXTERNAL TABLE users(key string, state string, country string, country_id int) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.columns.mapping" = "info:state,info:country,info:country_id" -); +) +TBLPROPERTIES ("external.table.purge" = "true"); -CREATE TABLE states(key string, name string) +CREATE EXTERNAL TABLE states(key string, name string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.columns.mapping" = "state:name" -); +) +TBLPROPERTIES ("external.table.purge" = "true"); -CREATE TABLE countries(key string, name string, country string, country_id int) +CREATE EXTERNAL TABLE countries(key string, name string, country string, country_id int) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.columns.mapping" = "info:name,info:country,info:country_id" -); +) +TBLPROPERTIES ("external.table.purge" = "true"); INSERT OVERWRITE TABLE users SELECT 'user1', 'IA', 'USA', 0 FROM src WHERE key=100; @@ -64,13 +67,15 @@ DROP TABLE users; DROP TABLE states; DROP TABLE countries; -CREATE TABLE users(key int, userid int, username string, created int) +CREATE EXTERNAL TABLE users(key int, userid int, username string, created int) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' -WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,f:userid,f:nickname,f:created"); +WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,f:userid,f:nickname,f:created") +TBLPROPERTIES ("external.table.purge" = "true"); -CREATE TABLE users_level(key int, userid int, level int) +CREATE EXTERNAL TABLE users_level(key int, userid int, level int) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' -WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,f:userid,f:level"); +WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,f:userid,f:level") +TBLPROPERTIES ("external.table.purge" = "true"); -- HIVE-1903: the problem fixed here showed up even without any data, -- so no need to load any to test it diff --git a/hbase-handler/src/test/queries/positive/hbase_null_first_col.q b/hbase-handler/src/test/queries/positive/hbase_null_first_col.q index 0d9ff56d97..5c5c311633 100644 --- a/hbase-handler/src/test/queries/positive/hbase_null_first_col.q +++ b/hbase-handler/src/test/queries/positive/hbase_null_first_col.q @@ -4,11 +4,12 @@ DROP TABLE hbase_null; CREATE TABLE src_null(a STRING, b STRING, c STRING, d STRING) STORED AS TEXTFILE; LOAD DATA LOCAL INPATH '../../data/files/null.txt' INTO TABLE src_null; -CREATE TABLE hbase_null(key string, col1 string, col2 string) +CREATE EXTERNAL TABLE hbase_null(key string, col1 string, col2 string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.columns.mapping" = ":key,cf1:c1,cf1:c2" -); +) +TBLPROPERTIES ("external.table.purge" = "true"); SELECT d, a, c FROM src_null; diff --git a/hbase-handler/src/test/queries/positive/hbase_ppd_join.q b/hbase-handler/src/test/queries/positive/hbase_ppd_join.q index 2436c19fed..94a397cca8 100644 --- a/hbase-handler/src/test/queries/positive/hbase_ppd_join.q +++ b/hbase-handler/src/test/queries/positive/hbase_ppd_join.q @@ -4,11 +4,11 @@ drop table if exists hive1_tbl_data_hbase2; drop view if exists hive1_view_data_hbase1; drop view if exists hive1_view_data_hbase2; -CREATE TABLE hive1_tbl_data_hbase1 (COLUMID string,COLUMN_FN string,COLUMN_LN string,EMAIL string,COL_UPDATED_DATE timestamp, PK_COLUM string) +CREATE EXTERNAL TABLE hive1_tbl_data_hbase1 (COLUMID string,COLUMN_FN string,COLUMN_LN string,EMAIL string,COL_UPDATED_DATE timestamp, PK_COLUM string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES("hbase.columns.mapping" = "default:COLUMID,default:COLUMN_FN,default:COLUMN_LN,default:EMAIL,default:COL_UPDATED_DATE,:key" ) -; +TBLPROPERTIES ("external.table.purge" = "true"); --create hive view for the above hive table 1 CREATE VIEW hive1_view_data_hbase1 @@ -25,11 +25,11 @@ AND COL_UPDATED_DATE IS NOT NULL insert into table hive1_tbl_data_hbase1 select '00001','john','doe','john@hotmail.com','2014-01-01 12:01:02','4000-10000' from src where key = 100; --create hive hbase table 2 -CREATE TABLE hive1_tbl_data_hbase2 (COLUMID string,COLUMN_FN string,COLUMN_LN string,EMAIL string,COL_UPDATED_DATE timestamp, PK_COLUM string) +CREATE EXTERNAL TABLE hive1_tbl_data_hbase2 (COLUMID string,COLUMN_FN string,COLUMN_LN string,EMAIL string,COL_UPDATED_DATE timestamp, PK_COLUM string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES("hbase.columns.mapping" = "default:COLUMID,default:COLUMN_FN,default:COLUMN_LN,default:EMAIL,default:COL_UPDATED_DATE,:key" ) -; +TBLPROPERTIES ("external.table.purge" = "true"); --create hive view for the above hive hbase table 2 CREATE VIEW hive1_view_data_hbase2 diff --git a/hbase-handler/src/test/queries/positive/hbase_ppd_key_range.q b/hbase-handler/src/test/queries/positive/hbase_ppd_key_range.q index 59e724dceb..ffba3f5d04 100644 --- a/hbase-handler/src/test/queries/positive/hbase_ppd_key_range.q +++ b/hbase-handler/src/test/queries/positive/hbase_ppd_key_range.q @@ -1,6 +1,7 @@ -CREATE TABLE hbase_pushdown(key string, value string) +CREATE EXTERNAL TABLE hbase_pushdown(key string, value string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' -WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,cf:string"); +WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,cf:string") +TBLPROPERTIES ("external.table.purge" = "true"); INSERT OVERWRITE TABLE hbase_pushdown SELECT cast(key as string), value diff --git a/hbase-handler/src/test/queries/positive/hbase_pushdown.q b/hbase-handler/src/test/queries/positive/hbase_pushdown.q index 0d29c829d0..24c1fac6da 100644 --- a/hbase-handler/src/test/queries/positive/hbase_pushdown.q +++ b/hbase-handler/src/test/queries/positive/hbase_pushdown.q @@ -1,6 +1,7 @@ -CREATE TABLE hbase_pushdown(key int, value string) +CREATE EXTERNAL TABLE hbase_pushdown(key int, value string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' -WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,cf:string"); +WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,cf:string") +TBLPROPERTIES ("external.table.purge" = "true"); INSERT OVERWRITE TABLE hbase_pushdown SELECT * diff --git a/hbase-handler/src/test/queries/positive/hbase_queries.q b/hbase-handler/src/test/queries/positive/hbase_queries.q index 4604f3e6f5..6c3d7cd5c0 100644 --- a/hbase-handler/src/test/queries/positive/hbase_queries.q +++ b/hbase-handler/src/test/queries/positive/hbase_queries.q @@ -1,8 +1,8 @@ DROP TABLE hbase_table_1; -CREATE TABLE hbase_table_1(key int comment 'It is a column key', value string comment 'It is the column string value') +CREATE EXTERNAL TABLE hbase_table_1(key int comment 'It is a column key', value string comment 'It is the column string value') STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = "cf:string") -TBLPROPERTIES ("hbase.table.name" = "hbase_table_0"); +TBLPROPERTIES ("hbase.table.name" = "hbase_table_0", "external.table.purge" = "true"); DESCRIBE EXTENDED hbase_table_1; @@ -52,9 +52,10 @@ ON (x.key = Y.key) ORDER BY key,value; DROP TABLE empty_hbase_table; -CREATE TABLE empty_hbase_table(key int, value string) +CREATE EXTERNAL TABLE empty_hbase_table(key int, value string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' -WITH SERDEPROPERTIES ("hbase.columns.mapping" = "cf:string"); +WITH SERDEPROPERTIES ("hbase.columns.mapping" = "cf:string") +TBLPROPERTIES ("external.table.purge" = "true"); DROP TABLE empty_normal_table; CREATE TABLE empty_normal_table(key int, value string); @@ -64,11 +65,12 @@ select * from (select count(1) c from empty_normal_table union all select count( select * from (select count(1) c from src union all select count(1) as c from empty_hbase_table) x order by c; select * from (select count(1) c from src union all select count(1) as c from hbase_table_1) x order by c; -CREATE TABLE hbase_table_3(key int, value string, count int) +CREATE EXTERNAL TABLE hbase_table_3(key int, value string, count int) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.columns.mapping" = "cf:val,cf2:count" -); +) +TBLPROPERTIES ("external.table.purge" = "true"); EXPLAIN INSERT OVERWRITE TABLE hbase_table_3 @@ -92,11 +94,12 @@ select * from hbase_table_3 order by key, value limit 5; select key, count from hbase_table_3 order by key, count desc limit 5; DROP TABLE hbase_table_4; -CREATE TABLE hbase_table_4(key int, value1 string, value2 int, value3 int) +CREATE EXTERNAL TABLE hbase_table_4(key int, value1 string, value2 int, value3 int) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.columns.mapping" = "a:b,a:c,d:e" -); +) +TBLPROPERTIES ("external.table.purge" = "true"); INSERT OVERWRITE TABLE hbase_table_4 SELECT key, value, key+1, key+2 FROM src WHERE key=98 OR key=100; @@ -112,22 +115,24 @@ TBLPROPERTIES ("hbase.table.name" = "hbase_table_4"); SELECT * FROM hbase_table_5 ORDER BY key; DROP TABLE hbase_table_6; -CREATE TABLE hbase_table_6(key int, value map) +CREATE EXTERNAL TABLE hbase_table_6(key int, value map) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.columns.mapping" = ":key,cf:" -); +) +TBLPROPERTIES ("external.table.purge" = "true"); INSERT OVERWRITE TABLE hbase_table_6 SELECT key, map(value, key) FROM src WHERE key=98 OR key=100; SELECT * FROM hbase_table_6 ORDER BY key; DROP TABLE hbase_table_7; -CREATE TABLE hbase_table_7(value map, key int) +CREATE EXTERNAL TABLE hbase_table_7(value map, key int) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.columns.mapping" = "cf:,:key" -); +) +TBLPROPERTIES ("external.table.purge" = "true"); INSERT OVERWRITE TABLE hbase_table_7 SELECT map(value, key, upper(value), key+1), key FROM src WHERE key=98 OR key=100; @@ -137,11 +142,12 @@ SELECT * FROM hbase_table_7 ORDER BY key; set hive.hbase.wal.enabled=false; DROP TABLE hbase_table_8; -CREATE TABLE hbase_table_8(key int, value1 string, value2 int, value3 int) +CREATE EXTERNAL TABLE hbase_table_8(key int, value1 string, value2 int, value3 int) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.columns.mapping" = "a:b,a:c,d:e" -); +) +TBLPROPERTIES ("external.table.purge" = "true"); INSERT OVERWRITE TABLE hbase_table_8 SELECT key, value, key+1, key+2 FROM src WHERE key=98 OR key=100; @@ -165,9 +171,10 @@ SELECT COUNT(*) FROM hbase_table_1_like; SHOW CREATE TABLE hbase_table_1_like; DROP TABLE IF EXISTS hbase_table_9; -CREATE TABLE hbase_table_9 (id bigint, data map, str string) +CREATE EXTERNAL TABLE hbase_table_9 (id bigint, data map, str string) stored by 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' -with serdeproperties ("hbase.columns.mapping" = ":key,cf:map_col#s:s,cf:str_col"); +with serdeproperties ("hbase.columns.mapping" = ":key,cf:map_col#s:s,cf:str_col") +TBLPROPERTIES ("external.table.purge" = "true"); insert overwrite table hbase_table_9 select 1 as id, map('abcd', null) as data , null as str from src limit 1; insert into table hbase_table_9 select 2 as id, map('efgh', null) as data , '1234' as str from src limit 1; @@ -177,9 +184,10 @@ insert into table hbase_table_9 select 5 as id, map('key1',null, 'key2', 'avalue select * from hbase_table_9; DROP TABLE IF EXISTS hbase_table_10; -CREATE TABLE hbase_table_10 (id bigint, data map, str string) +CREATE EXTERNAL TABLE hbase_table_10 (id bigint, data map, str string) stored by 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' -with serdeproperties ("hbase.columns.mapping" = ":key,cf:map_col2,cf:str2_col"); +with serdeproperties ("hbase.columns.mapping" = ":key,cf:map_col2,cf:str2_col") +TBLPROPERTIES ("external.table.purge" = "true"); set hive.cbo.enable=false; insert overwrite table hbase_table_10 select 1 as id, map(10, cast(null as int)) as data , null as str from src limit 1; insert into table hbase_table_10 select 2 as id, map(20, cast(null as int)) as data , '1234' as str from src limit 1; @@ -190,16 +198,18 @@ select * from hbase_table_10; DROP TABLE IF EXISTS hbase_table_11; -CREATE TABLE hbase_table_11(id INT, map_column STRUCT) +CREATE EXTERNAL TABLE hbase_table_11(id INT, map_column STRUCT) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' -WITH SERDEPROPERTIES ('hbase.columns.mapping'=':key,id:id'); +WITH SERDEPROPERTIES ('hbase.columns.mapping'=':key,id:id') +TBLPROPERTIES ("external.table.purge" = "true"); INSERT INTO hbase_table_11 SELECT 2,NAMED_STRUCT("s_int",CAST(NULL AS INT),"s_string","s1","s_date",CAST('2018-03-12' AS DATE)) FROM src LIMIT 1; select * from hbase_table_11; DROP TABLE IF EXISTS hbase_table_12; -CREATE TABLE hbase_table_12(id INT, list_column ARRAY ) +CREATE EXTERNAL TABLE hbase_table_12(id INT, list_column ARRAY ) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' -WITH SERDEPROPERTIES ('hbase.columns.mapping'=':key,id:id'); +WITH SERDEPROPERTIES ('hbase.columns.mapping'=':key,id:id') +TBLPROPERTIES ("external.table.purge" = "true"); INSERT INTO hbase_table_12 SELECT 2, ARRAY("a", CAST (NULL AS STRING), "b") FROM src LIMIT 1; select * from hbase_table_12; diff --git a/hbase-handler/src/test/queries/positive/hbase_scan_params.q b/hbase-handler/src/test/queries/positive/hbase_scan_params.q index 14d7d15831..7bea8f033b 100644 --- a/hbase-handler/src/test/queries/positive/hbase_scan_params.q +++ b/hbase-handler/src/test/queries/positive/hbase_scan_params.q @@ -1,7 +1,8 @@ -CREATE TABLE hbase_pushdown(key int, value string) +CREATE EXTERNAL TABLE hbase_pushdown(key int, value string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,cf:string", -"hbase.scan.cache" = "500", "hbase.scan.cacheblocks" = "true", "hbase.scan.batch" = "1"); +"hbase.scan.cache" = "500", "hbase.scan.cacheblocks" = "true", "hbase.scan.batch" = "1") +TBLPROPERTIES ("external.table.purge" = "true"); INSERT OVERWRITE TABLE hbase_pushdown SELECT * FROM src; diff --git a/hbase-handler/src/test/queries/positive/hbase_single_sourced_multi_insert.q b/hbase-handler/src/test/queries/positive/hbase_single_sourced_multi_insert.q index 96fec0ecc6..4ca02970f6 100644 --- a/hbase-handler/src/test/queries/positive/hbase_single_sourced_multi_insert.q +++ b/hbase-handler/src/test/queries/positive/hbase_single_sourced_multi_insert.q @@ -1,8 +1,9 @@ -- HIVE-4375 Single sourced multi insert consists of native and non-native table mixed throws NPE CREATE TABLE src_x1(key string, value string); -CREATE TABLE src_x2(key string, value string) +CREATE EXTERNAL TABLE src_x2(key string, value string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' -WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key, cf:value"); +WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key, cf:value") +TBLPROPERTIES ("external.table.purge" = "true"); explain from src a diff --git a/hbase-handler/src/test/queries/positive/hbase_timestamp.q b/hbase-handler/src/test/queries/positive/hbase_timestamp.q index 6ae2c30596..19db69d9a8 100644 --- a/hbase-handler/src/test/queries/positive/hbase_timestamp.q +++ b/hbase-handler/src/test/queries/positive/hbase_timestamp.q @@ -1,22 +1,25 @@ DROP TABLE hbase_table; -CREATE TABLE hbase_table (key string, value string, `time` timestamp) +CREATE EXTERNAL TABLE hbase_table (key string, value string, `time` timestamp) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' - WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,cf:string,:timestamp"); + WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,cf:string,:timestamp") + TBLPROPERTIES ("external.table.purge" = "true"); DESC extended hbase_table; FROM src INSERT OVERWRITE TABLE hbase_table SELECT key, value, "2012-02-23 10:14:52" WHERE (key % 17) = 0; SELECT * FROM hbase_table; DROP TABLE hbase_table; -CREATE TABLE hbase_table (key string, value string, `time` bigint) +CREATE EXTERNAL TABLE hbase_table (key string, value string, `time` bigint) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' - WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,cf:string,:timestamp"); + WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,cf:string,:timestamp") + TBLPROPERTIES ("external.table.purge" = "true"); FROM src INSERT OVERWRITE TABLE hbase_table SELECT key, value, 1329959754000 WHERE (key % 17) = 0; SELECT key, value, cast(`time` as timestamp) FROM hbase_table; DROP TABLE hbase_table; -CREATE TABLE hbase_table (key string, value string, `time` bigint) +CREATE EXTERNAL TABLE hbase_table (key string, value string, `time` bigint) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' - WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,cf:string,:timestamp"); + WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,cf:string,:timestamp") + TBLPROPERTIES ("external.table.purge" = "true"); insert overwrite table hbase_table select key,value,ts FROM ( select key, value, 100000000000 as ts from src WHERE (key % 33) = 0 @@ -41,9 +44,10 @@ SELECT key, value, cast(`time` as timestamp) FROM hbase_table WHERE key > 100 AN SELECT key, value, cast(`time` as timestamp) FROM hbase_table WHERE key > 100 AND key < 400 AND `time` >= 200000000000; DROP TABLE hbase_table; -CREATE TABLE hbase_table(key string, value map, `time` timestamp) +CREATE EXTERNAL TABLE hbase_table(key string, value map, `time` timestamp) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' -WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,cf:,:timestamp"); +WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,cf:,:timestamp") +TBLPROPERTIES ("external.table.purge" = "true"); FROM src INSERT OVERWRITE TABLE hbase_table SELECT key, MAP("name", CONCAT(value, " Jr")), "2012-02-23 10:14:52" WHERE (key % 17) = 0; FROM src INSERT INTO TABLE hbase_table SELECT key, MAP("age", '40'), "2015-12-12 12:12:12" WHERE (key % 17) = 0; FROM src INSERT INTO TABLE hbase_table SELECT key, MAP("name", value), "2000-01-01 01:01:01" WHERE (key % 17) = 0; diff --git a/hbase-handler/src/test/queries/positive/hbase_timestamp_format.q b/hbase-handler/src/test/queries/positive/hbase_timestamp_format.q index a8d5501c5e..7ed0495757 100644 --- a/hbase-handler/src/test/queries/positive/hbase_timestamp_format.q +++ b/hbase-handler/src/test/queries/positive/hbase_timestamp_format.q @@ -1,8 +1,8 @@ -create table hbase_str(rowkey string,mytime string,mystr string) +create external table hbase_str(rowkey string,mytime string,mystr string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ('hbase.columns.mapping' = 'm:mytime,m:mystr') - TBLPROPERTIES ('hbase.table.name' = 'hbase_ts'); + TBLPROPERTIES ('hbase.table.name' = 'hbase_ts', 'external.table.purge' = 'true'); describe hbase_str; insert overwrite table hbase_str select key, '2001-02-03-04.05.06.123456', value from src limit 3; diff --git a/hbase-handler/src/test/queries/positive/hbase_viewjoins.q b/hbase-handler/src/test/queries/positive/hbase_viewjoins.q index 5c98903b39..f18a099a1b 100644 --- a/hbase-handler/src/test/queries/positive/hbase_viewjoins.q +++ b/hbase-handler/src/test/queries/positive/hbase_viewjoins.q @@ -2,7 +2,7 @@ DROP VIEW IF EXISTS VIEW_HBASE_TABLE_TEST_2; DROP VIEW IF EXISTS VIEW_HBASE_TABLE_TEST_1; DROP TABLE IF EXISTS HBASE_TABLE_TEST_2; DROP TABLE IF EXISTS HBASE_TABLE_TEST_1; -CREATE TABLE HBASE_TABLE_TEST_1( +CREATE EXTERNAL TABLE HBASE_TABLE_TEST_1( cvalue string , pk string, ccount int ) @@ -17,11 +17,12 @@ WITH SERDEPROPERTIES ( 'serialization.format'='1') TBLPROPERTIES ( 'hbase.table.name'='hbase_table_test_1', - 'serialization.null.format'='' ); + 'serialization.null.format'='', + 'external.table.purge' = 'true'); CREATE VIEW VIEW_HBASE_TABLE_TEST_1 AS SELECT hbase_table_test_1.cvalue,hbase_table_test_1.pk,hbase_table_test_1.ccount FROM hbase_table_test_1 WHERE hbase_table_test_1.ccount IS NOT NULL; -CREATE TABLE HBASE_TABLE_TEST_2( +CREATE EXTERNAL TABLE HBASE_TABLE_TEST_2( cvalue string , pk string , ccount int ) @@ -36,7 +37,8 @@ WITH SERDEPROPERTIES ( 'serialization.format'='1') TBLPROPERTIES ( 'hbase.table.name'='hbase_table_test_2', - 'serialization.null.format'=''); + 'serialization.null.format'='', + 'external.table.purge' = 'true'); CREATE VIEW VIEW_HBASE_TABLE_TEST_2 AS SELECT hbase_table_test_2.cvalue,hbase_table_test_2.pk,hbase_table_test_2.ccount FROM hbase_table_test_2 WHERE hbase_table_test_2.pk >='3-0000h-0' AND hbase_table_test_2.pk <= '3-0000h-g' AND diff --git a/hbase-handler/src/test/queries/positive/hbasestats.q b/hbase-handler/src/test/queries/positive/hbasestats.q index 52e11c9757..98933ed364 100644 --- a/hbase-handler/src/test/queries/positive/hbasestats.q +++ b/hbase-handler/src/test/queries/positive/hbasestats.q @@ -1,10 +1,11 @@ DROP TABLE users; -CREATE TABLE users(key string, state string, country string, country_id int) +CREATE EXTERNAL TABLE users(key string, state string, country string, country_id int) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.columns.mapping" = "info:state,info:country,info:country_id" -); +) +TBLPROPERTIES ("external.table.purge" = "true"); desc formatted users; diff --git a/hbase-handler/src/test/queries/positive/ppd_key_ranges.q b/hbase-handler/src/test/queries/positive/ppd_key_ranges.q index 0497d25ffa..186f742469 100644 --- a/hbase-handler/src/test/queries/positive/ppd_key_ranges.q +++ b/hbase-handler/src/test/queries/positive/ppd_key_ranges.q @@ -1,6 +1,7 @@ -CREATE TABLE hbase_ppd_keyrange(key int, value string) +CREATE EXTERNAL TABLE hbase_ppd_keyrange(key int, value string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' -WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key#binary,cf:string"); +WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key#binary,cf:string") +TBLPROPERTIES ("external.table.purge" = "true"); INSERT OVERWRITE TABLE hbase_ppd_keyrange SELECT * diff --git a/hbase-handler/src/test/results/negative/cascade_dbdrop.q.out b/hbase-handler/src/test/results/negative/cascade_dbdrop.q.out index cef7a06093..803e35e406 100644 --- a/hbase-handler/src/test/results/negative/cascade_dbdrop.q.out +++ b/hbase-handler/src/test/results/negative/cascade_dbdrop.q.out @@ -4,17 +4,17 @@ PREHOOK: Output: database:hbaseDB POSTHOOK: query: CREATE DATABASE hbaseDB POSTHOOK: type: CREATEDATABASE POSTHOOK: Output: database:hbaseDB -PREHOOK: query: CREATE TABLE hbaseDB.hbase_table_0(key int, value string) +PREHOOK: query: CREATE EXTERNAL TABLE hbaseDB.hbase_table_0(key int, value string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,cf:string") -TBLPROPERTIES ("hbase.table.name" = "hbase_table_0") +TBLPROPERTIES ("hbase.table.name" = "hbase_table_0", "external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:hbasedb PREHOOK: Output: hbaseDB@hbase_table_0 -POSTHOOK: query: CREATE TABLE hbaseDB.hbase_table_0(key int, value string) +POSTHOOK: query: CREATE EXTERNAL TABLE hbaseDB.hbase_table_0(key int, value string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,cf:string") -TBLPROPERTIES ("hbase.table.name" = "hbase_table_0") +TBLPROPERTIES ("hbase.table.name" = "hbase_table_0", "external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:hbasedb POSTHOOK: Output: hbaseDB@hbase_table_0 diff --git a/hbase-handler/src/test/results/negative/generatehfiles_require_family_path.q.out b/hbase-handler/src/test/results/negative/generatehfiles_require_family_path.q.out index 52ac38fa3e..aab4cf9c4c 100644 --- a/hbase-handler/src/test/results/negative/generatehfiles_require_family_path.q.out +++ b/hbase-handler/src/test/results/negative/generatehfiles_require_family_path.q.out @@ -2,15 +2,17 @@ PREHOOK: query: DROP TABLE IF EXISTS hbase_bulk PREHOOK: type: DROPTABLE POSTHOOK: query: DROP TABLE IF EXISTS hbase_bulk POSTHOOK: type: DROPTABLE -PREHOOK: query: CREATE TABLE hbase_bulk (key INT, value STRING) +PREHOOK: query: CREATE EXTERNAL TABLE hbase_bulk (key INT, value STRING) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ('hbase.columns.mapping' = ':key,cf:string') +TBLPROPERTIES ("external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@hbase_bulk -POSTHOOK: query: CREATE TABLE hbase_bulk (key INT, value STRING) +POSTHOOK: query: CREATE EXTERNAL TABLE hbase_bulk (key INT, value STRING) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ('hbase.columns.mapping' = ':key,cf:string') +TBLPROPERTIES ("external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@hbase_bulk diff --git a/hbase-handler/src/test/results/negative/hbase_ddl.q.out b/hbase-handler/src/test/results/negative/hbase_ddl.q.out index fde2fc55e5..e4c146b8a7 100644 --- a/hbase-handler/src/test/results/negative/hbase_ddl.q.out +++ b/hbase-handler/src/test/results/negative/hbase_ddl.q.out @@ -2,17 +2,17 @@ PREHOOK: query: DROP TABLE hbase_table_1 PREHOOK: type: DROPTABLE POSTHOOK: query: DROP TABLE hbase_table_1 POSTHOOK: type: DROPTABLE -PREHOOK: query: CREATE TABLE hbase_table_1(key int comment 'It is a column key', value string comment 'It is the column string value') +PREHOOK: query: CREATE EXTERNAL TABLE hbase_table_1(key int comment 'It is a column key', value string comment 'It is the column string value') STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = "cf:string") -TBLPROPERTIES ("hbase.table.name" = "hbase_table_0") +TBLPROPERTIES ("hbase.table.name" = "hbase_table_0", "external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@hbase_table_1 -POSTHOOK: query: CREATE TABLE hbase_table_1(key int comment 'It is a column key', value string comment 'It is the column string value') +POSTHOOK: query: CREATE EXTERNAL TABLE hbase_table_1(key int comment 'It is a column key', value string comment 'It is the column string value') STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = "cf:string") -TBLPROPERTIES ("hbase.table.name" = "hbase_table_0") +TBLPROPERTIES ("hbase.table.name" = "hbase_table_0", "external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@hbase_table_1 diff --git a/hbase-handler/src/test/results/positive/external_table_ppd.q.out b/hbase-handler/src/test/results/positive/external_table_ppd.q.out index aed43cf64d..68329208cb 100644 --- a/hbase-handler/src/test/results/positive/external_table_ppd.q.out +++ b/hbase-handler/src/test/results/positive/external_table_ppd.q.out @@ -2,7 +2,7 @@ PREHOOK: query: DROP TABLE t_hbase PREHOOK: type: DROPTABLE POSTHOOK: query: DROP TABLE t_hbase POSTHOOK: type: DROPTABLE -PREHOOK: query: CREATE TABLE t_hbase(key STRING, +PREHOOK: query: CREATE EXTERNAL TABLE t_hbase(key STRING, tinyint_col TINYINT, smallint_col SMALLINT, int_col INT, @@ -13,11 +13,12 @@ PREHOOK: query: CREATE TABLE t_hbase(key STRING, STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = "cf:binarykey#-,cf:binarybyte#-,cf:binaryshort#-,:key#-,cf:binarylong#-,cf:binaryfloat#-,cf:binarydouble#-,cf:binaryboolean#-") TBLPROPERTIES ("hbase.table.name" = "t_hive", - "hbase.table.default.storage.type" = "binary") + "hbase.table.default.storage.type" = "binary", + "external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@t_hbase -POSTHOOK: query: CREATE TABLE t_hbase(key STRING, +POSTHOOK: query: CREATE EXTERNAL TABLE t_hbase(key STRING, tinyint_col TINYINT, smallint_col SMALLINT, int_col INT, @@ -28,7 +29,8 @@ POSTHOOK: query: CREATE TABLE t_hbase(key STRING, STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = "cf:binarykey#-,cf:binarybyte#-,cf:binaryshort#-,:key#-,cf:binarylong#-,cf:binaryfloat#-,cf:binarydouble#-,cf:binaryboolean#-") TBLPROPERTIES ("hbase.table.name" = "t_hive", - "hbase.table.default.storage.type" = "binary") + "hbase.table.default.storage.type" = "binary", + "external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@t_hbase @@ -53,10 +55,12 @@ Database: default #### A masked pattern was here #### Retention: 0 #### A masked pattern was here #### -Table Type: MANAGED_TABLE +Table Type: EXTERNAL_TABLE Table Parameters: COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"bigint_col\":\"true\",\"boolean_col\":\"true\",\"double_col\":\"true\",\"float_col\":\"true\",\"int_col\":\"true\",\"key\":\"true\",\"smallint_col\":\"true\",\"tinyint_col\":\"true\"}} + EXTERNAL TRUE bucketing_version 2 + external.table.purge true hbase.table.default.storage.type binary hbase.table.name t_hive numFiles 0 diff --git a/hbase-handler/src/test/results/positive/hbase_binary_binary.q.out b/hbase-handler/src/test/results/positive/hbase_binary_binary.q.out index e04227fbd2..debe89c760 100644 --- a/hbase-handler/src/test/results/positive/hbase_binary_binary.q.out +++ b/hbase-handler/src/test/results/positive/hbase_binary_binary.q.out @@ -2,19 +2,21 @@ PREHOOK: query: drop table if exists testhbaseb PREHOOK: type: DROPTABLE POSTHOOK: query: drop table if exists testhbaseb POSTHOOK: type: DROPTABLE -PREHOOK: query: CREATE TABLE testhbaseb (key int, val binary) +PREHOOK: query: CREATE EXTERNAL TABLE testhbaseb (key int, val binary) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.columns.mapping" = ":key,cf:val#b" ) +TBLPROPERTIES ("external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@testhbaseb -POSTHOOK: query: CREATE TABLE testhbaseb (key int, val binary) +POSTHOOK: query: CREATE EXTERNAL TABLE testhbaseb (key int, val binary) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.columns.mapping" = ":key,cf:val#b" ) +TBLPROPERTIES ("external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@testhbaseb diff --git a/hbase-handler/src/test/results/positive/hbase_binary_map_queries.q.out b/hbase-handler/src/test/results/positive/hbase_binary_map_queries.q.out index feaca4bf38..57fea2cf2f 100644 --- a/hbase-handler/src/test/results/positive/hbase_binary_map_queries.q.out +++ b/hbase-handler/src/test/results/positive/hbase_binary_map_queries.q.out @@ -2,7 +2,7 @@ PREHOOK: query: DROP TABLE hbase_src PREHOOK: type: DROPTABLE POSTHOOK: query: DROP TABLE hbase_src POSTHOOK: type: DROPTABLE -PREHOOK: query: CREATE TABLE hbase_src(key STRING, +PREHOOK: query: CREATE EXTERNAL TABLE hbase_src(key STRING, tinyint_col TINYINT, smallint_col SMALLINT, int_col INT, @@ -10,10 +10,11 @@ PREHOOK: query: CREATE TABLE hbase_src(key STRING, float_col FLOAT, double_col DOUBLE, string_col STRING) +TBLPROPERTIES ("external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@hbase_src -POSTHOOK: query: CREATE TABLE hbase_src(key STRING, +POSTHOOK: query: CREATE EXTERNAL TABLE hbase_src(key STRING, tinyint_col TINYINT, smallint_col SMALLINT, int_col INT, @@ -21,6 +22,7 @@ POSTHOOK: query: CREATE TABLE hbase_src(key STRING, float_col FLOAT, double_col DOUBLE, string_col STRING) +TBLPROPERTIES ("external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@hbase_src @@ -50,7 +52,7 @@ PREHOOK: query: DROP TABLE t_hbase_maps PREHOOK: type: DROPTABLE POSTHOOK: query: DROP TABLE t_hbase_maps POSTHOOK: type: DROPTABLE -PREHOOK: query: CREATE TABLE t_hbase_maps(key STRING, +PREHOOK: query: CREATE EXTERNAL TABLE t_hbase_maps(key STRING, tinyint_map_col MAP, smallint_map_col MAP, int_map_col MAP, @@ -61,11 +63,11 @@ PREHOOK: query: CREATE TABLE t_hbase_maps(key STRING, boolean_map_col MAP) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping"=":key,cf-tinyint:,cf-smallint:,cf-int:,cf-bigint:,cf-float:,cf-double:,cf-string:,cf-boolean:") -TBLPROPERTIES ("hbase.table.name"="t_hive_maps") +TBLPROPERTIES ("hbase.table.name"="t_hive_maps", "external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@t_hbase_maps -POSTHOOK: query: CREATE TABLE t_hbase_maps(key STRING, +POSTHOOK: query: CREATE EXTERNAL TABLE t_hbase_maps(key STRING, tinyint_map_col MAP, smallint_map_col MAP, int_map_col MAP, @@ -76,7 +78,7 @@ POSTHOOK: query: CREATE TABLE t_hbase_maps(key STRING, boolean_map_col MAP) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping"=":key,cf-tinyint:,cf-smallint:,cf-int:,cf-bigint:,cf-float:,cf-double:,cf-string:,cf-boolean:") -TBLPROPERTIES ("hbase.table.name"="t_hive_maps") +TBLPROPERTIES ("hbase.table.name"="t_hive_maps", "external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@t_hbase_maps @@ -314,7 +316,7 @@ PREHOOK: query: DROP TABLE t_hbase_maps_1 PREHOOK: type: DROPTABLE POSTHOOK: query: DROP TABLE t_hbase_maps_1 POSTHOOK: type: DROPTABLE -PREHOOK: query: CREATE TABLE t_hbase_maps_1(key STRING, +PREHOOK: query: CREATE EXTERNAL TABLE t_hbase_maps_1(key STRING, tinyint_map_col MAP, smallint_map_col MAP, int_map_col MAP, @@ -325,11 +327,11 @@ PREHOOK: query: CREATE TABLE t_hbase_maps_1(key STRING, boolean_map_col MAP) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping"=":key#b,cf-tinyint:#b:b,cf-smallint:#b:b,cf-int:#b:b,cf-bigint:#b:b,cf-float:#b:b,cf-double:#b:b,cf-string:#b:b,cf-boolean:#b:b") -TBLPROPERTIES ("hbase.table.name"="t_hive_maps_1") +TBLPROPERTIES ("hbase.table.name"="t_hive_maps_1", "external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@t_hbase_maps_1 -POSTHOOK: query: CREATE TABLE t_hbase_maps_1(key STRING, +POSTHOOK: query: CREATE EXTERNAL TABLE t_hbase_maps_1(key STRING, tinyint_map_col MAP, smallint_map_col MAP, int_map_col MAP, @@ -340,7 +342,7 @@ POSTHOOK: query: CREATE TABLE t_hbase_maps_1(key STRING, boolean_map_col MAP) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping"=":key#b,cf-tinyint:#b:b,cf-smallint:#b:b,cf-int:#b:b,cf-bigint:#b:b,cf-float:#b:b,cf-double:#b:b,cf-string:#b:b,cf-boolean:#b:b") -TBLPROPERTIES ("hbase.table.name"="t_hive_maps_1") +TBLPROPERTIES ("hbase.table.name"="t_hive_maps_1", "external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@t_hbase_maps_1 diff --git a/hbase-handler/src/test/results/positive/hbase_binary_map_queries_prefix.q.out b/hbase-handler/src/test/results/positive/hbase_binary_map_queries_prefix.q.out index f6432b32e2..946d72228d 100644 --- a/hbase-handler/src/test/results/positive/hbase_binary_map_queries_prefix.q.out +++ b/hbase-handler/src/test/results/positive/hbase_binary_map_queries_prefix.q.out @@ -2,7 +2,7 @@ PREHOOK: query: DROP TABLE hbase_src PREHOOK: type: DROPTABLE POSTHOOK: query: DROP TABLE hbase_src POSTHOOK: type: DROPTABLE -PREHOOK: query: CREATE TABLE hbase_src(key STRING, +PREHOOK: query: CREATE EXTERNAL TABLE hbase_src(key STRING, tinyint_col TINYINT, smallint_col SMALLINT, int_col INT, @@ -10,10 +10,11 @@ PREHOOK: query: CREATE TABLE hbase_src(key STRING, float_col FLOAT, double_col DOUBLE, string_col STRING) +TBLPROPERTIES ("external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@hbase_src -POSTHOOK: query: CREATE TABLE hbase_src(key STRING, +POSTHOOK: query: CREATE EXTERNAL TABLE hbase_src(key STRING, tinyint_col TINYINT, smallint_col SMALLINT, int_col INT, @@ -21,6 +22,7 @@ POSTHOOK: query: CREATE TABLE hbase_src(key STRING, float_col FLOAT, double_col DOUBLE, string_col STRING) +TBLPROPERTIES ("external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@hbase_src @@ -50,21 +52,21 @@ PREHOOK: query: DROP TABLE t_hbase_maps PREHOOK: type: DROPTABLE POSTHOOK: query: DROP TABLE t_hbase_maps POSTHOOK: type: DROPTABLE -PREHOOK: query: CREATE TABLE t_hbase_maps(key STRING, +PREHOOK: query: CREATE EXTERNAL TABLE t_hbase_maps(key STRING, string_map_col MAP, simple_string_col STRING) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping"=":key,cf-string:,cf-string:simple_string_col") -TBLPROPERTIES ("hbase.table.name"="t_hive_maps") +TBLPROPERTIES ("hbase.table.name"="t_hive_maps", "external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@t_hbase_maps -POSTHOOK: query: CREATE TABLE t_hbase_maps(key STRING, +POSTHOOK: query: CREATE EXTERNAL TABLE t_hbase_maps(key STRING, string_map_col MAP, simple_string_col STRING) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping"=":key,cf-string:,cf-string:simple_string_col") -TBLPROPERTIES ("hbase.table.name"="t_hive_maps") +TBLPROPERTIES ("hbase.table.name"="t_hive_maps", "external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@t_hbase_maps diff --git a/hbase-handler/src/test/results/positive/hbase_binary_storage_queries.q.out b/hbase-handler/src/test/results/positive/hbase_binary_storage_queries.q.out index 172db75438..1209c880f7 100644 --- a/hbase-handler/src/test/results/positive/hbase_binary_storage_queries.q.out +++ b/hbase-handler/src/test/results/positive/hbase_binary_storage_queries.q.out @@ -2,7 +2,7 @@ PREHOOK: query: DROP TABLE t_hbase PREHOOK: type: DROPTABLE POSTHOOK: query: DROP TABLE t_hbase POSTHOOK: type: DROPTABLE -PREHOOK: query: CREATE TABLE t_hbase(key STRING, +PREHOOK: query: CREATE EXTERNAL TABLE t_hbase(key STRING, tinyint_col TINYINT, smallint_col SMALLINT, int_col INT, @@ -13,11 +13,12 @@ PREHOOK: query: CREATE TABLE t_hbase(key STRING, STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key#-,cf:binarybyte#-,cf:binaryshort#-,cf:binaryint#-,cf:binarylong#-,cf:binaryfloat#-,cf:binarydouble#-,cf:binaryboolean#-") TBLPROPERTIES ("hbase.table.name" = "t_hive", - "hbase.table.default.storage.type" = "binary") + "hbase.table.default.storage.type" = "binary", + "external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@t_hbase -POSTHOOK: query: CREATE TABLE t_hbase(key STRING, +POSTHOOK: query: CREATE EXTERNAL TABLE t_hbase(key STRING, tinyint_col TINYINT, smallint_col SMALLINT, int_col INT, @@ -28,7 +29,8 @@ POSTHOOK: query: CREATE TABLE t_hbase(key STRING, STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key#-,cf:binarybyte#-,cf:binaryshort#-,cf:binaryint#-,cf:binarylong#-,cf:binaryfloat#-,cf:binarydouble#-,cf:binaryboolean#-") TBLPROPERTIES ("hbase.table.name" = "t_hive", - "hbase.table.default.storage.type" = "binary") + "hbase.table.default.storage.type" = "binary", + "external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@t_hbase @@ -53,10 +55,12 @@ Database: default #### A masked pattern was here #### Retention: 0 #### A masked pattern was here #### -Table Type: MANAGED_TABLE +Table Type: EXTERNAL_TABLE Table Parameters: COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"bigint_col\":\"true\",\"boolean_col\":\"true\",\"double_col\":\"true\",\"float_col\":\"true\",\"int_col\":\"true\",\"key\":\"true\",\"smallint_col\":\"true\",\"tinyint_col\":\"true\"}} + EXTERNAL TRUE bucketing_version 2 + external.table.purge true hbase.table.default.storage.type binary hbase.table.name t_hive numFiles 0 @@ -338,7 +342,7 @@ PREHOOK: query: DROP TABLE t_hbase_2 PREHOOK: type: DROPTABLE POSTHOOK: query: DROP TABLE t_hbase_2 POSTHOOK: type: DROPTABLE -PREHOOK: query: CREATE TABLE t_hbase_2(key STRING, +PREHOOK: query: CREATE EXTERNAL TABLE t_hbase_2(key STRING, tinyint_col TINYINT, smallint_col SMALLINT, int_col INT, @@ -348,11 +352,11 @@ PREHOOK: query: CREATE TABLE t_hbase_2(key STRING, boolean_col BOOLEAN) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key#-,cf:binarybyte#-,cf:binaryshort#-,cf:binaryint#-,cf:binarylong#-,cf:binaryfloat#-,cf:binarydouble#-,cf:binaryboolean#-") -TBLPROPERTIES ("hbase.table.name" = "t_hive_2") +TBLPROPERTIES ("hbase.table.name" = "t_hive_2", "external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@t_hbase_2 -POSTHOOK: query: CREATE TABLE t_hbase_2(key STRING, +POSTHOOK: query: CREATE EXTERNAL TABLE t_hbase_2(key STRING, tinyint_col TINYINT, smallint_col SMALLINT, int_col INT, @@ -362,7 +366,7 @@ POSTHOOK: query: CREATE TABLE t_hbase_2(key STRING, boolean_col BOOLEAN) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key#-,cf:binarybyte#-,cf:binaryshort#-,cf:binaryint#-,cf:binarylong#-,cf:binaryfloat#-,cf:binarydouble#-,cf:binaryboolean#-") -TBLPROPERTIES ("hbase.table.name" = "t_hive_2") +TBLPROPERTIES ("hbase.table.name" = "t_hive_2", "external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@t_hbase_2 @@ -586,7 +590,8 @@ STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key#-,cf:binarybyte#-,cf:binaryshort#-,cf:binaryint#-,cf:binarylong#-,cf:binaryfloat#-,cf:binarydouble#-,cf:binaryboolean#-") TBLPROPERTIES ( "hbase.table.name" = "t_hive_2", -"hbase.table.default.storage.type" = "binary") +"hbase.table.default.storage.type" = "binary", +"external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@t_hbase_4 @@ -602,7 +607,8 @@ STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key#-,cf:binarybyte#-,cf:binaryshort#-,cf:binaryint#-,cf:binarylong#-,cf:binaryfloat#-,cf:binarydouble#-,cf:binaryboolean#-") TBLPROPERTIES ( "hbase.table.name" = "t_hive_2", -"hbase.table.default.storage.type" = "binary") +"hbase.table.default.storage.type" = "binary", +"external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@t_hbase_4 diff --git a/hbase-handler/src/test/results/positive/hbase_custom_key.q.out b/hbase-handler/src/test/results/positive/hbase_custom_key.q.out index e5bc947a79..bd3bfcf1cc 100644 --- a/hbase-handler/src/test/results/positive/hbase_custom_key.q.out +++ b/hbase-handler/src/test/results/positive/hbase_custom_key.q.out @@ -1,18 +1,20 @@ -PREHOOK: query: CREATE TABLE hbase_ck_1(key struct, value string) +PREHOOK: query: CREATE EXTERNAL TABLE hbase_ck_1(key struct, value string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.table.name" = "hbase_custom", "hbase.columns.mapping" = ":key,cf:string", "hbase.composite.key.factory"="org.apache.hadoop.hive.hbase.SampleHBaseKeyFactory") +TBLPROPERTIES ("external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@hbase_ck_1 -POSTHOOK: query: CREATE TABLE hbase_ck_1(key struct, value string) +POSTHOOK: query: CREATE EXTERNAL TABLE hbase_ck_1(key struct, value string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.table.name" = "hbase_custom", "hbase.columns.mapping" = ":key,cf:string", "hbase.composite.key.factory"="org.apache.hadoop.hive.hbase.SampleHBaseKeyFactory") +TBLPROPERTIES ("external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@hbase_ck_1 diff --git a/hbase-handler/src/test/results/positive/hbase_custom_key2.q.out b/hbase-handler/src/test/results/positive/hbase_custom_key2.q.out index 5d381e773a..8355a438cc 100644 --- a/hbase-handler/src/test/results/positive/hbase_custom_key2.q.out +++ b/hbase-handler/src/test/results/positive/hbase_custom_key2.q.out @@ -1,18 +1,20 @@ -PREHOOK: query: CREATE TABLE hbase_ck_4(key struct, value string) +PREHOOK: query: CREATE EXTERNAL TABLE hbase_ck_4(key struct, value string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.table.name" = "hbase_custom2", "hbase.columns.mapping" = ":key,cf:string", "hbase.composite.key.factory"="org.apache.hadoop.hive.hbase.SampleHBaseKeyFactory2") +TBLPROPERTIES ("external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@hbase_ck_4 -POSTHOOK: query: CREATE TABLE hbase_ck_4(key struct, value string) +POSTHOOK: query: CREATE EXTERNAL TABLE hbase_ck_4(key struct, value string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.table.name" = "hbase_custom2", "hbase.columns.mapping" = ":key,cf:string", "hbase.composite.key.factory"="org.apache.hadoop.hive.hbase.SampleHBaseKeyFactory2") +TBLPROPERTIES ("external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@hbase_ck_4 diff --git a/hbase-handler/src/test/results/positive/hbase_custom_key3.q.out b/hbase-handler/src/test/results/positive/hbase_custom_key3.q.out index 1b042fd840..68781901a5 100644 --- a/hbase-handler/src/test/results/positive/hbase_custom_key3.q.out +++ b/hbase-handler/src/test/results/positive/hbase_custom_key3.q.out @@ -1,18 +1,20 @@ -PREHOOK: query: CREATE TABLE hbase_ck_5(key struct, value string) +PREHOOK: query: CREATE EXTERNAL TABLE hbase_ck_5(key struct, value string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.table.name" = "hbase_custom3", "hbase.columns.mapping" = ":key,cf:string", "hbase.composite.key.factory"="org.apache.hadoop.hive.hbase.SampleHBaseKeyFactory3") +TBLPROPERTIES ("external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@hbase_ck_5 -POSTHOOK: query: CREATE TABLE hbase_ck_5(key struct, value string) +POSTHOOK: query: CREATE EXTERNAL TABLE hbase_ck_5(key struct, value string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.table.name" = "hbase_custom3", "hbase.columns.mapping" = ":key,cf:string", "hbase.composite.key.factory"="org.apache.hadoop.hive.hbase.SampleHBaseKeyFactory3") +TBLPROPERTIES ("external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@hbase_ck_5 diff --git a/hbase-handler/src/test/results/positive/hbase_ddl.q.out b/hbase-handler/src/test/results/positive/hbase_ddl.q.out index 9d16034f06..4cb2083054 100644 --- a/hbase-handler/src/test/results/positive/hbase_ddl.q.out +++ b/hbase-handler/src/test/results/positive/hbase_ddl.q.out @@ -2,17 +2,17 @@ PREHOOK: query: DROP TABLE hbase_table_1 PREHOOK: type: DROPTABLE POSTHOOK: query: DROP TABLE hbase_table_1 POSTHOOK: type: DROPTABLE -PREHOOK: query: CREATE TABLE hbase_table_1(key int comment 'It is a column key', value string comment 'It is the column string value') +PREHOOK: query: CREATE EXTERNAL TABLE hbase_table_1(key int comment 'It is a column key', value string comment 'It is the column string value') STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = "cf:string") -TBLPROPERTIES ("hbase.table.name" = "hbase_table_0") +TBLPROPERTIES ("hbase.table.name" = "hbase_table_0", "external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@hbase_table_1 -POSTHOOK: query: CREATE TABLE hbase_table_1(key int comment 'It is a column key', value string comment 'It is the column string value') +POSTHOOK: query: CREATE EXTERNAL TABLE hbase_table_1(key int comment 'It is a column key', value string comment 'It is the column string value') STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = "cf:string") -TBLPROPERTIES ("hbase.table.name" = "hbase_table_0") +TBLPROPERTIES ("hbase.table.name" = "hbase_table_0", "external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@hbase_table_1 @@ -110,9 +110,11 @@ Database: default #### A masked pattern was here #### Retention: 0 #### A masked pattern was here #### -Table Type: MANAGED_TABLE +Table Type: EXTERNAL_TABLE Table Parameters: + EXTERNAL TRUE bucketing_version 2 + external.table.purge true hbase.mapred.output.outputtable kkk hbase.table.name hbase_table_0 #### A masked pattern was here #### @@ -157,9 +159,11 @@ Database: default #### A masked pattern was here #### Retention: 0 #### A masked pattern was here #### -Table Type: MANAGED_TABLE +Table Type: EXTERNAL_TABLE Table Parameters: + EXTERNAL TRUE bucketing_version 2 + external.table.purge true hbase.table.name hbase_table_0 #### A masked pattern was here #### numFiles 0 diff --git a/hbase-handler/src/test/results/positive/hbase_decimal_decimal.q.out b/hbase-handler/src/test/results/positive/hbase_decimal_decimal.q.out index 6bd4f7386c..aa9a5acffc 100644 --- a/hbase-handler/src/test/results/positive/hbase_decimal_decimal.q.out +++ b/hbase-handler/src/test/results/positive/hbase_decimal_decimal.q.out @@ -1,4 +1,4 @@ -PREHOOK: query: CREATE TABLE testhbase_decimal ( +PREHOOK: query: CREATE EXTERNAL TABLE testhbase_decimal ( id int, balance decimal(15,2)) ROW FORMAT DELIMITED @@ -6,10 +6,11 @@ COLLECTION ITEMS TERMINATED BY '~' STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.columns.mapping"=":key,cf:balance#b") +TBLPROPERTIES ("external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@testhbase_decimal -POSTHOOK: query: CREATE TABLE testhbase_decimal ( +POSTHOOK: query: CREATE EXTERNAL TABLE testhbase_decimal ( id int, balance decimal(15,2)) ROW FORMAT DELIMITED @@ -17,6 +18,7 @@ COLLECTION ITEMS TERMINATED BY '~' STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.columns.mapping"=":key,cf:balance#b") +TBLPROPERTIES ("external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@testhbase_decimal diff --git a/hbase-handler/src/test/results/positive/hbase_handler_bulk.q.out b/hbase-handler/src/test/results/positive/hbase_handler_bulk.q.out index de1c13976f..0b8c853840 100644 --- a/hbase-handler/src/test/results/positive/hbase_handler_bulk.q.out +++ b/hbase-handler/src/test/results/positive/hbase_handler_bulk.q.out @@ -2,17 +2,17 @@ PREHOOK: query: drop table if exists hb_target PREHOOK: type: DROPTABLE POSTHOOK: query: drop table if exists hb_target POSTHOOK: type: DROPTABLE -PREHOOK: query: create table hb_target(key int, val string) +PREHOOK: query: create external table hb_target(key int, val string) stored by 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' with serdeproperties ('hbase.columns.mapping' = ':key,cf:val') -tblproperties ('hbase.table.name' = 'positive_hbase_handler_bulk') +tblproperties ('hbase.table.name' = 'positive_hbase_handler_bulk', 'external.table.purge' = 'true') PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@hb_target -POSTHOOK: query: create table hb_target(key int, val string) +POSTHOOK: query: create external table hb_target(key int, val string) stored by 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' with serdeproperties ('hbase.columns.mapping' = ':key,cf:val') -tblproperties ('hbase.table.name' = 'positive_hbase_handler_bulk') +tblproperties ('hbase.table.name' = 'positive_hbase_handler_bulk', 'external.table.purge' = 'true') POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@hb_target @@ -33,17 +33,17 @@ POSTHOOK: type: DROPTABLE POSTHOOK: Input: default@hb_target POSTHOOK: Output: default@hb_target #### A masked pattern was here #### -PREHOOK: query: create table hb_target(key int, val string) +PREHOOK: query: create external table hb_target(key int, val string) stored by 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' with serdeproperties ('hbase.columns.mapping' = ':key,cf:val') -tblproperties ('hbase.table.name' = 'positive_hbase_handler_bulk') +tblproperties ('hbase.table.name' = 'positive_hbase_handler_bulk', 'external.table.purge' = 'true') PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@hb_target -POSTHOOK: query: create table hb_target(key int, val string) +POSTHOOK: query: create external table hb_target(key int, val string) stored by 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' with serdeproperties ('hbase.columns.mapping' = ':key,cf:val') -tblproperties ('hbase.table.name' = 'positive_hbase_handler_bulk') +tblproperties ('hbase.table.name' = 'positive_hbase_handler_bulk', 'external.table.purge' = 'true') POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@hb_target diff --git a/hbase-handler/src/test/results/positive/hbase_joins.q.out b/hbase-handler/src/test/results/positive/hbase_joins.q.out index 697675e4c8..a591c4e4b0 100644 --- a/hbase-handler/src/test/results/positive/hbase_joins.q.out +++ b/hbase-handler/src/test/results/positive/hbase_joins.q.out @@ -14,51 +14,57 @@ PREHOOK: query: DROP TABLE users_level PREHOOK: type: DROPTABLE POSTHOOK: query: DROP TABLE users_level POSTHOOK: type: DROPTABLE -PREHOOK: query: CREATE TABLE users(key string, state string, country string, country_id int) +PREHOOK: query: CREATE EXTERNAL TABLE users(key string, state string, country string, country_id int) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.columns.mapping" = "info:state,info:country,info:country_id" ) +TBLPROPERTIES ("external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@users -POSTHOOK: query: CREATE TABLE users(key string, state string, country string, country_id int) +POSTHOOK: query: CREATE EXTERNAL TABLE users(key string, state string, country string, country_id int) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.columns.mapping" = "info:state,info:country,info:country_id" ) +TBLPROPERTIES ("external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@users -PREHOOK: query: CREATE TABLE states(key string, name string) +PREHOOK: query: CREATE EXTERNAL TABLE states(key string, name string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.columns.mapping" = "state:name" ) +TBLPROPERTIES ("external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@states -POSTHOOK: query: CREATE TABLE states(key string, name string) +POSTHOOK: query: CREATE EXTERNAL TABLE states(key string, name string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.columns.mapping" = "state:name" ) +TBLPROPERTIES ("external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@states -PREHOOK: query: CREATE TABLE countries(key string, name string, country string, country_id int) +PREHOOK: query: CREATE EXTERNAL TABLE countries(key string, name string, country string, country_id int) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.columns.mapping" = "info:name,info:country,info:country_id" ) +TBLPROPERTIES ("external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@countries -POSTHOOK: query: CREATE TABLE countries(key string, name string, country string, country_id int) +POSTHOOK: query: CREATE EXTERNAL TABLE countries(key string, name string, country string, country_id int) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.columns.mapping" = "info:name,info:country,info:country_id" ) +TBLPROPERTIES ("external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@countries @@ -218,27 +224,31 @@ POSTHOOK: query: DROP TABLE countries POSTHOOK: type: DROPTABLE POSTHOOK: Input: default@countries POSTHOOK: Output: default@countries -PREHOOK: query: CREATE TABLE users(key int, userid int, username string, created int) +PREHOOK: query: CREATE EXTERNAL TABLE users(key int, userid int, username string, created int) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,f:userid,f:nickname,f:created") +TBLPROPERTIES ("external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@users -POSTHOOK: query: CREATE TABLE users(key int, userid int, username string, created int) +POSTHOOK: query: CREATE EXTERNAL TABLE users(key int, userid int, username string, created int) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,f:userid,f:nickname,f:created") +TBLPROPERTIES ("external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@users -PREHOOK: query: CREATE TABLE users_level(key int, userid int, level int) +PREHOOK: query: CREATE EXTERNAL TABLE users_level(key int, userid int, level int) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,f:userid,f:level") +TBLPROPERTIES ("external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@users_level -POSTHOOK: query: CREATE TABLE users_level(key int, userid int, level int) +POSTHOOK: query: CREATE EXTERNAL TABLE users_level(key int, userid int, level int) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,f:userid,f:level") +TBLPROPERTIES ("external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@users_level diff --git a/hbase-handler/src/test/results/positive/hbase_null_first_col.q.out b/hbase-handler/src/test/results/positive/hbase_null_first_col.q.out index bb4491bf08..47f56fcd79 100644 --- a/hbase-handler/src/test/results/positive/hbase_null_first_col.q.out +++ b/hbase-handler/src/test/results/positive/hbase_null_first_col.q.out @@ -22,19 +22,21 @@ POSTHOOK: query: LOAD DATA LOCAL INPATH '../../data/files/null.txt' INTO TABLE s POSTHOOK: type: LOAD #### A masked pattern was here #### POSTHOOK: Output: default@src_null -PREHOOK: query: CREATE TABLE hbase_null(key string, col1 string, col2 string) +PREHOOK: query: CREATE EXTERNAL TABLE hbase_null(key string, col1 string, col2 string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.columns.mapping" = ":key,cf1:c1,cf1:c2" ) +TBLPROPERTIES ("external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@hbase_null -POSTHOOK: query: CREATE TABLE hbase_null(key string, col1 string, col2 string) +POSTHOOK: query: CREATE EXTERNAL TABLE hbase_null(key string, col1 string, col2 string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.columns.mapping" = ":key,cf1:c1,cf1:c2" ) +TBLPROPERTIES ("external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@hbase_null diff --git a/hbase-handler/src/test/results/positive/hbase_ppd_join.q.out b/hbase-handler/src/test/results/positive/hbase_ppd_join.q.out index 0744cafc21..47327e5140 100644 --- a/hbase-handler/src/test/results/positive/hbase_ppd_join.q.out +++ b/hbase-handler/src/test/results/positive/hbase_ppd_join.q.out @@ -14,17 +14,19 @@ PREHOOK: query: drop view if exists hive1_view_data_hbase2 PREHOOK: type: DROPVIEW POSTHOOK: query: drop view if exists hive1_view_data_hbase2 POSTHOOK: type: DROPVIEW -PREHOOK: query: CREATE TABLE hive1_tbl_data_hbase1 (COLUMID string,COLUMN_FN string,COLUMN_LN string,EMAIL string,COL_UPDATED_DATE timestamp, PK_COLUM string) +PREHOOK: query: CREATE EXTERNAL TABLE hive1_tbl_data_hbase1 (COLUMID string,COLUMN_FN string,COLUMN_LN string,EMAIL string,COL_UPDATED_DATE timestamp, PK_COLUM string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES("hbase.columns.mapping" = "default:COLUMID,default:COLUMN_FN,default:COLUMN_LN,default:EMAIL,default:COL_UPDATED_DATE,:key" ) +TBLPROPERTIES ("external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@hive1_tbl_data_hbase1 -POSTHOOK: query: CREATE TABLE hive1_tbl_data_hbase1 (COLUMID string,COLUMN_FN string,COLUMN_LN string,EMAIL string,COL_UPDATED_DATE timestamp, PK_COLUM string) +POSTHOOK: query: CREATE EXTERNAL TABLE hive1_tbl_data_hbase1 (COLUMID string,COLUMN_FN string,COLUMN_LN string,EMAIL string,COL_UPDATED_DATE timestamp, PK_COLUM string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES("hbase.columns.mapping" = "default:COLUMID,default:COLUMN_FN,default:COLUMN_LN,default:EMAIL,default:COL_UPDATED_DATE,:key" ) +TBLPROPERTIES ("external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@hive1_tbl_data_hbase1 @@ -64,17 +66,19 @@ POSTHOOK: query: insert into table hive1_tbl_data_hbase1 select '00001','john',' POSTHOOK: type: QUERY POSTHOOK: Input: default@src POSTHOOK: Output: default@hive1_tbl_data_hbase1 -PREHOOK: query: CREATE TABLE hive1_tbl_data_hbase2 (COLUMID string,COLUMN_FN string,COLUMN_LN string,EMAIL string,COL_UPDATED_DATE timestamp, PK_COLUM string) +PREHOOK: query: CREATE EXTERNAL TABLE hive1_tbl_data_hbase2 (COLUMID string,COLUMN_FN string,COLUMN_LN string,EMAIL string,COL_UPDATED_DATE timestamp, PK_COLUM string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES("hbase.columns.mapping" = "default:COLUMID,default:COLUMN_FN,default:COLUMN_LN,default:EMAIL,default:COL_UPDATED_DATE,:key" ) +TBLPROPERTIES ("external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@hive1_tbl_data_hbase2 -POSTHOOK: query: CREATE TABLE hive1_tbl_data_hbase2 (COLUMID string,COLUMN_FN string,COLUMN_LN string,EMAIL string,COL_UPDATED_DATE timestamp, PK_COLUM string) +POSTHOOK: query: CREATE EXTERNAL TABLE hive1_tbl_data_hbase2 (COLUMID string,COLUMN_FN string,COLUMN_LN string,EMAIL string,COL_UPDATED_DATE timestamp, PK_COLUM string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES("hbase.columns.mapping" = "default:COLUMID,default:COLUMN_FN,default:COLUMN_LN,default:EMAIL,default:COL_UPDATED_DATE,:key" ) +TBLPROPERTIES ("external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@hive1_tbl_data_hbase2 diff --git a/hbase-handler/src/test/results/positive/hbase_ppd_key_range.q.out b/hbase-handler/src/test/results/positive/hbase_ppd_key_range.q.out index 75294ed4db..4d44d1d33b 100644 --- a/hbase-handler/src/test/results/positive/hbase_ppd_key_range.q.out +++ b/hbase-handler/src/test/results/positive/hbase_ppd_key_range.q.out @@ -1,12 +1,14 @@ -PREHOOK: query: CREATE TABLE hbase_pushdown(key string, value string) +PREHOOK: query: CREATE EXTERNAL TABLE hbase_pushdown(key string, value string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,cf:string") +TBLPROPERTIES ("external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@hbase_pushdown -POSTHOOK: query: CREATE TABLE hbase_pushdown(key string, value string) +POSTHOOK: query: CREATE EXTERNAL TABLE hbase_pushdown(key string, value string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,cf:string") +TBLPROPERTIES ("external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@hbase_pushdown diff --git a/hbase-handler/src/test/results/positive/hbase_pushdown.q.out b/hbase-handler/src/test/results/positive/hbase_pushdown.q.out index 28161645fc..909a5fab25 100644 --- a/hbase-handler/src/test/results/positive/hbase_pushdown.q.out +++ b/hbase-handler/src/test/results/positive/hbase_pushdown.q.out @@ -1,12 +1,14 @@ -PREHOOK: query: CREATE TABLE hbase_pushdown(key int, value string) +PREHOOK: query: CREATE EXTERNAL TABLE hbase_pushdown(key int, value string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,cf:string") +TBLPROPERTIES ("external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@hbase_pushdown -POSTHOOK: query: CREATE TABLE hbase_pushdown(key int, value string) +POSTHOOK: query: CREATE EXTERNAL TABLE hbase_pushdown(key int, value string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,cf:string") +TBLPROPERTIES ("external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@hbase_pushdown diff --git a/hbase-handler/src/test/results/positive/hbase_queries.q.out b/hbase-handler/src/test/results/positive/hbase_queries.q.out index eb91bf9fc6..a746dccd83 100644 --- a/hbase-handler/src/test/results/positive/hbase_queries.q.out +++ b/hbase-handler/src/test/results/positive/hbase_queries.q.out @@ -2,17 +2,17 @@ PREHOOK: query: DROP TABLE hbase_table_1 PREHOOK: type: DROPTABLE POSTHOOK: query: DROP TABLE hbase_table_1 POSTHOOK: type: DROPTABLE -PREHOOK: query: CREATE TABLE hbase_table_1(key int comment 'It is a column key', value string comment 'It is the column string value') +PREHOOK: query: CREATE EXTERNAL TABLE hbase_table_1(key int comment 'It is a column key', value string comment 'It is the column string value') STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = "cf:string") -TBLPROPERTIES ("hbase.table.name" = "hbase_table_0") +TBLPROPERTIES ("hbase.table.name" = "hbase_table_0", "external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@hbase_table_1 -POSTHOOK: query: CREATE TABLE hbase_table_1(key int comment 'It is a column key', value string comment 'It is the column string value') +POSTHOOK: query: CREATE EXTERNAL TABLE hbase_table_1(key int comment 'It is a column key', value string comment 'It is the column string value') STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = "cf:string") -TBLPROPERTIES ("hbase.table.name" = "hbase_table_0") +TBLPROPERTIES ("hbase.table.name" = "hbase_table_0", "external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@hbase_table_1 @@ -393,15 +393,17 @@ PREHOOK: query: DROP TABLE empty_hbase_table PREHOOK: type: DROPTABLE POSTHOOK: query: DROP TABLE empty_hbase_table POSTHOOK: type: DROPTABLE -PREHOOK: query: CREATE TABLE empty_hbase_table(key int, value string) +PREHOOK: query: CREATE EXTERNAL TABLE empty_hbase_table(key int, value string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = "cf:string") +TBLPROPERTIES ("external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@empty_hbase_table -POSTHOOK: query: CREATE TABLE empty_hbase_table(key int, value string) +POSTHOOK: query: CREATE EXTERNAL TABLE empty_hbase_table(key int, value string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = "cf:string") +TBLPROPERTIES ("external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@empty_hbase_table @@ -465,19 +467,21 @@ POSTHOOK: Input: default@src #### A masked pattern was here #### 155 500 -PREHOOK: query: CREATE TABLE hbase_table_3(key int, value string, count int) +PREHOOK: query: CREATE EXTERNAL TABLE hbase_table_3(key int, value string, count int) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.columns.mapping" = "cf:val,cf2:count" ) +TBLPROPERTIES ("external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@hbase_table_3 -POSTHOOK: query: CREATE TABLE hbase_table_3(key int, value string, count int) +POSTHOOK: query: CREATE EXTERNAL TABLE hbase_table_3(key int, value string, count int) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.columns.mapping" = "cf:val,cf2:count" ) +TBLPROPERTIES ("external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@hbase_table_3 @@ -663,19 +667,21 @@ PREHOOK: query: DROP TABLE hbase_table_4 PREHOOK: type: DROPTABLE POSTHOOK: query: DROP TABLE hbase_table_4 POSTHOOK: type: DROPTABLE -PREHOOK: query: CREATE TABLE hbase_table_4(key int, value1 string, value2 int, value3 int) +PREHOOK: query: CREATE EXTERNAL TABLE hbase_table_4(key int, value1 string, value2 int, value3 int) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.columns.mapping" = "a:b,a:c,d:e" ) +TBLPROPERTIES ("external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@hbase_table_4 -POSTHOOK: query: CREATE TABLE hbase_table_4(key int, value1 string, value2 int, value3 int) +POSTHOOK: query: CREATE EXTERNAL TABLE hbase_table_4(key int, value1 string, value2 int, value3 int) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.columns.mapping" = "a:b,a:c,d:e" ) +TBLPROPERTIES ("external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@hbase_table_4 @@ -731,19 +737,21 @@ PREHOOK: query: DROP TABLE hbase_table_6 PREHOOK: type: DROPTABLE POSTHOOK: query: DROP TABLE hbase_table_6 POSTHOOK: type: DROPTABLE -PREHOOK: query: CREATE TABLE hbase_table_6(key int, value map) +PREHOOK: query: CREATE EXTERNAL TABLE hbase_table_6(key int, value map) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.columns.mapping" = ":key,cf:" ) +TBLPROPERTIES ("external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@hbase_table_6 -POSTHOOK: query: CREATE TABLE hbase_table_6(key int, value map) +POSTHOOK: query: CREATE EXTERNAL TABLE hbase_table_6(key int, value map) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.columns.mapping" = ":key,cf:" ) +TBLPROPERTIES ("external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@hbase_table_6 @@ -771,19 +779,21 @@ PREHOOK: query: DROP TABLE hbase_table_7 PREHOOK: type: DROPTABLE POSTHOOK: query: DROP TABLE hbase_table_7 POSTHOOK: type: DROPTABLE -PREHOOK: query: CREATE TABLE hbase_table_7(value map, key int) +PREHOOK: query: CREATE EXTERNAL TABLE hbase_table_7(value map, key int) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.columns.mapping" = "cf:,:key" ) +TBLPROPERTIES ("external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@hbase_table_7 -POSTHOOK: query: CREATE TABLE hbase_table_7(value map, key int) +POSTHOOK: query: CREATE EXTERNAL TABLE hbase_table_7(value map, key int) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.columns.mapping" = "cf:,:key" ) +TBLPROPERTIES ("external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@hbase_table_7 @@ -813,19 +823,21 @@ PREHOOK: query: DROP TABLE hbase_table_8 PREHOOK: type: DROPTABLE POSTHOOK: query: DROP TABLE hbase_table_8 POSTHOOK: type: DROPTABLE -PREHOOK: query: CREATE TABLE hbase_table_8(key int, value1 string, value2 int, value3 int) +PREHOOK: query: CREATE EXTERNAL TABLE hbase_table_8(key int, value1 string, value2 int, value3 int) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.columns.mapping" = "a:b,a:c,d:e" ) +TBLPROPERTIES ("external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@hbase_table_8 -POSTHOOK: query: CREATE TABLE hbase_table_8(key int, value1 string, value2 int, value3 int) +POSTHOOK: query: CREATE EXTERNAL TABLE hbase_table_8(key int, value1 string, value2 int, value3 int) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.columns.mapping" = "a:b,a:c,d:e" ) +TBLPROPERTIES ("external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@hbase_table_8 @@ -955,15 +967,17 @@ PREHOOK: query: DROP TABLE IF EXISTS hbase_table_9 PREHOOK: type: DROPTABLE POSTHOOK: query: DROP TABLE IF EXISTS hbase_table_9 POSTHOOK: type: DROPTABLE -PREHOOK: query: CREATE TABLE hbase_table_9 (id bigint, data map, str string) +PREHOOK: query: CREATE EXTERNAL TABLE hbase_table_9 (id bigint, data map, str string) stored by 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' with serdeproperties ("hbase.columns.mapping" = ":key,cf:map_col#s:s,cf:str_col") +TBLPROPERTIES ("external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@hbase_table_9 -POSTHOOK: query: CREATE TABLE hbase_table_9 (id bigint, data map, str string) +POSTHOOK: query: CREATE EXTERNAL TABLE hbase_table_9 (id bigint, data map, str string) stored by 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' with serdeproperties ("hbase.columns.mapping" = ":key,cf:map_col#s:s,cf:str_col") +TBLPROPERTIES ("external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@hbase_table_9 @@ -1024,15 +1038,17 @@ PREHOOK: query: DROP TABLE IF EXISTS hbase_table_10 PREHOOK: type: DROPTABLE POSTHOOK: query: DROP TABLE IF EXISTS hbase_table_10 POSTHOOK: type: DROPTABLE -PREHOOK: query: CREATE TABLE hbase_table_10 (id bigint, data map, str string) +PREHOOK: query: CREATE EXTERNAL TABLE hbase_table_10 (id bigint, data map, str string) stored by 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' with serdeproperties ("hbase.columns.mapping" = ":key,cf:map_col2,cf:str2_col") +TBLPROPERTIES ("external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@hbase_table_10 -POSTHOOK: query: CREATE TABLE hbase_table_10 (id bigint, data map, str string) +POSTHOOK: query: CREATE EXTERNAL TABLE hbase_table_10 (id bigint, data map, str string) stored by 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' with serdeproperties ("hbase.columns.mapping" = ":key,cf:map_col2,cf:str2_col") +TBLPROPERTIES ("external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@hbase_table_10 @@ -1093,15 +1109,17 @@ PREHOOK: query: DROP TABLE IF EXISTS hbase_table_11 PREHOOK: type: DROPTABLE POSTHOOK: query: DROP TABLE IF EXISTS hbase_table_11 POSTHOOK: type: DROPTABLE -PREHOOK: query: CREATE TABLE hbase_table_11(id INT, map_column STRUCT) +PREHOOK: query: CREATE EXTERNAL TABLE hbase_table_11(id INT, map_column STRUCT) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ('hbase.columns.mapping'=':key,id:id') +TBLPROPERTIES ("external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@hbase_table_11 -POSTHOOK: query: CREATE TABLE hbase_table_11(id INT, map_column STRUCT) +POSTHOOK: query: CREATE EXTERNAL TABLE hbase_table_11(id INT, map_column STRUCT) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ('hbase.columns.mapping'=':key,id:id') +TBLPROPERTIES ("external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@hbase_table_11 @@ -1126,15 +1144,17 @@ PREHOOK: query: DROP TABLE IF EXISTS hbase_table_12 PREHOOK: type: DROPTABLE POSTHOOK: query: DROP TABLE IF EXISTS hbase_table_12 POSTHOOK: type: DROPTABLE -PREHOOK: query: CREATE TABLE hbase_table_12(id INT, list_column ARRAY ) +PREHOOK: query: CREATE EXTERNAL TABLE hbase_table_12(id INT, list_column ARRAY ) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ('hbase.columns.mapping'=':key,id:id') +TBLPROPERTIES ("external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@hbase_table_12 -POSTHOOK: query: CREATE TABLE hbase_table_12(id INT, list_column ARRAY ) +POSTHOOK: query: CREATE EXTERNAL TABLE hbase_table_12(id INT, list_column ARRAY ) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ('hbase.columns.mapping'=':key,id:id') +TBLPROPERTIES ("external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@hbase_table_12 diff --git a/hbase-handler/src/test/results/positive/hbase_scan_params.q.out b/hbase-handler/src/test/results/positive/hbase_scan_params.q.out index 1ccaba0038..5ae8b1a73f 100644 --- a/hbase-handler/src/test/results/positive/hbase_scan_params.q.out +++ b/hbase-handler/src/test/results/positive/hbase_scan_params.q.out @@ -1,14 +1,16 @@ -PREHOOK: query: CREATE TABLE hbase_pushdown(key int, value string) +PREHOOK: query: CREATE EXTERNAL TABLE hbase_pushdown(key int, value string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,cf:string", "hbase.scan.cache" = "500", "hbase.scan.cacheblocks" = "true", "hbase.scan.batch" = "1") +TBLPROPERTIES ("external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@hbase_pushdown -POSTHOOK: query: CREATE TABLE hbase_pushdown(key int, value string) +POSTHOOK: query: CREATE EXTERNAL TABLE hbase_pushdown(key int, value string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,cf:string", "hbase.scan.cache" = "500", "hbase.scan.cacheblocks" = "true", "hbase.scan.batch" = "1") +TBLPROPERTIES ("external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@hbase_pushdown diff --git a/hbase-handler/src/test/results/positive/hbase_single_sourced_multi_insert.q.out b/hbase-handler/src/test/results/positive/hbase_single_sourced_multi_insert.q.out index 86a9feadfa..7982f22388 100644 --- a/hbase-handler/src/test/results/positive/hbase_single_sourced_multi_insert.q.out +++ b/hbase-handler/src/test/results/positive/hbase_single_sourced_multi_insert.q.out @@ -6,15 +6,17 @@ POSTHOOK: query: CREATE TABLE src_x1(key string, value string) POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@src_x1 -PREHOOK: query: CREATE TABLE src_x2(key string, value string) +PREHOOK: query: CREATE EXTERNAL TABLE src_x2(key string, value string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key, cf:value") +TBLPROPERTIES ("external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@src_x2 -POSTHOOK: query: CREATE TABLE src_x2(key string, value string) +POSTHOOK: query: CREATE EXTERNAL TABLE src_x2(key string, value string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key, cf:value") +TBLPROPERTIES ("external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@src_x2 diff --git a/hbase-handler/src/test/results/positive/hbase_timestamp.q.out b/hbase-handler/src/test/results/positive/hbase_timestamp.q.out index 6e7d7e6f6f..9318cd72cf 100644 --- a/hbase-handler/src/test/results/positive/hbase_timestamp.q.out +++ b/hbase-handler/src/test/results/positive/hbase_timestamp.q.out @@ -2,15 +2,17 @@ PREHOOK: query: DROP TABLE hbase_table PREHOOK: type: DROPTABLE POSTHOOK: query: DROP TABLE hbase_table POSTHOOK: type: DROPTABLE -PREHOOK: query: CREATE TABLE hbase_table (key string, value string, `time` timestamp) +PREHOOK: query: CREATE EXTERNAL TABLE hbase_table (key string, value string, `time` timestamp) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,cf:string,:timestamp") + TBLPROPERTIES ("external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@hbase_table -POSTHOOK: query: CREATE TABLE hbase_table (key string, value string, `time` timestamp) +POSTHOOK: query: CREATE EXTERNAL TABLE hbase_table (key string, value string, `time` timestamp) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,cf:string,:timestamp") + TBLPROPERTIES ("external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@hbase_table @@ -69,15 +71,17 @@ POSTHOOK: query: DROP TABLE hbase_table POSTHOOK: type: DROPTABLE POSTHOOK: Input: default@hbase_table POSTHOOK: Output: default@hbase_table -PREHOOK: query: CREATE TABLE hbase_table (key string, value string, `time` bigint) +PREHOOK: query: CREATE EXTERNAL TABLE hbase_table (key string, value string, `time` bigint) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,cf:string,:timestamp") + TBLPROPERTIES ("external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@hbase_table -POSTHOOK: query: CREATE TABLE hbase_table (key string, value string, `time` bigint) +POSTHOOK: query: CREATE EXTERNAL TABLE hbase_table (key string, value string, `time` bigint) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,cf:string,:timestamp") + TBLPROPERTIES ("external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@hbase_table @@ -125,15 +129,17 @@ POSTHOOK: query: DROP TABLE hbase_table POSTHOOK: type: DROPTABLE POSTHOOK: Input: default@hbase_table POSTHOOK: Output: default@hbase_table -PREHOOK: query: CREATE TABLE hbase_table (key string, value string, `time` bigint) +PREHOOK: query: CREATE EXTERNAL TABLE hbase_table (key string, value string, `time` bigint) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,cf:string,:timestamp") + TBLPROPERTIES ("external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@hbase_table -POSTHOOK: query: CREATE TABLE hbase_table (key string, value string, `time` bigint) +POSTHOOK: query: CREATE EXTERNAL TABLE hbase_table (key string, value string, `time` bigint) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,cf:string,:timestamp") + TBLPROPERTIES ("external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@hbase_table @@ -363,15 +369,17 @@ POSTHOOK: query: DROP TABLE hbase_table POSTHOOK: type: DROPTABLE POSTHOOK: Input: default@hbase_table POSTHOOK: Output: default@hbase_table -PREHOOK: query: CREATE TABLE hbase_table(key string, value map, `time` timestamp) +PREHOOK: query: CREATE EXTERNAL TABLE hbase_table(key string, value map, `time` timestamp) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,cf:,:timestamp") +TBLPROPERTIES ("external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@hbase_table -POSTHOOK: query: CREATE TABLE hbase_table(key string, value map, `time` timestamp) +POSTHOOK: query: CREATE EXTERNAL TABLE hbase_table(key string, value map, `time` timestamp) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key,cf:,:timestamp") +TBLPROPERTIES ("external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@hbase_table diff --git a/hbase-handler/src/test/results/positive/hbase_timestamp_format.q.out b/hbase-handler/src/test/results/positive/hbase_timestamp_format.q.out index a2a2f56691..8156724df8 100644 --- a/hbase-handler/src/test/results/positive/hbase_timestamp_format.q.out +++ b/hbase-handler/src/test/results/positive/hbase_timestamp_format.q.out @@ -1,14 +1,14 @@ -PREHOOK: query: create table hbase_str(rowkey string,mytime string,mystr string) +PREHOOK: query: create external table hbase_str(rowkey string,mytime string,mystr string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ('hbase.columns.mapping' = 'm:mytime,m:mystr') - TBLPROPERTIES ('hbase.table.name' = 'hbase_ts') + TBLPROPERTIES ('hbase.table.name' = 'hbase_ts', 'external.table.purge' = 'true') PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@hbase_str -POSTHOOK: query: create table hbase_str(rowkey string,mytime string,mystr string) +POSTHOOK: query: create external table hbase_str(rowkey string,mytime string,mystr string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ('hbase.columns.mapping' = 'm:mytime,m:mystr') - TBLPROPERTIES ('hbase.table.name' = 'hbase_ts') + TBLPROPERTIES ('hbase.table.name' = 'hbase_ts', 'external.table.purge' = 'true') POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@hbase_str diff --git a/hbase-handler/src/test/results/positive/hbase_viewjoins.q.out b/hbase-handler/src/test/results/positive/hbase_viewjoins.q.out index 95fcaa02e0..9eae99d51a 100644 --- a/hbase-handler/src/test/results/positive/hbase_viewjoins.q.out +++ b/hbase-handler/src/test/results/positive/hbase_viewjoins.q.out @@ -14,7 +14,7 @@ PREHOOK: query: DROP TABLE IF EXISTS HBASE_TABLE_TEST_1 PREHOOK: type: DROPTABLE POSTHOOK: query: DROP TABLE IF EXISTS HBASE_TABLE_TEST_1 POSTHOOK: type: DROPTABLE -PREHOOK: query: CREATE TABLE HBASE_TABLE_TEST_1( +PREHOOK: query: CREATE EXTERNAL TABLE HBASE_TABLE_TEST_1( cvalue string , pk string, ccount int ) @@ -29,11 +29,12 @@ WITH SERDEPROPERTIES ( 'serialization.format'='1') TBLPROPERTIES ( 'hbase.table.name'='hbase_table_test_1', - 'serialization.null.format'='' ) + 'serialization.null.format'='', + 'external.table.purge' = 'true') PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@HBASE_TABLE_TEST_1 -POSTHOOK: query: CREATE TABLE HBASE_TABLE_TEST_1( +POSTHOOK: query: CREATE EXTERNAL TABLE HBASE_TABLE_TEST_1( cvalue string , pk string, ccount int ) @@ -48,7 +49,8 @@ WITH SERDEPROPERTIES ( 'serialization.format'='1') TBLPROPERTIES ( 'hbase.table.name'='hbase_table_test_1', - 'serialization.null.format'='' ) + 'serialization.null.format'='', + 'external.table.purge' = 'true') POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@HBASE_TABLE_TEST_1 @@ -65,7 +67,7 @@ POSTHOOK: Output: default@VIEW_HBASE_TABLE_TEST_1 POSTHOOK: Lineage: VIEW_HBASE_TABLE_TEST_1.ccount SIMPLE [(hbase_table_test_1)hbase_table_test_1.FieldSchema(name:ccount, type:int, comment:), ] POSTHOOK: Lineage: VIEW_HBASE_TABLE_TEST_1.cvalue SIMPLE [(hbase_table_test_1)hbase_table_test_1.FieldSchema(name:cvalue, type:string, comment:), ] POSTHOOK: Lineage: VIEW_HBASE_TABLE_TEST_1.pk SIMPLE [(hbase_table_test_1)hbase_table_test_1.FieldSchema(name:pk, type:string, comment:), ] -PREHOOK: query: CREATE TABLE HBASE_TABLE_TEST_2( +PREHOOK: query: CREATE EXTERNAL TABLE HBASE_TABLE_TEST_2( cvalue string , pk string , ccount int ) @@ -80,11 +82,12 @@ WITH SERDEPROPERTIES ( 'serialization.format'='1') TBLPROPERTIES ( 'hbase.table.name'='hbase_table_test_2', - 'serialization.null.format'='') + 'serialization.null.format'='', + 'external.table.purge' = 'true') PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@HBASE_TABLE_TEST_2 -POSTHOOK: query: CREATE TABLE HBASE_TABLE_TEST_2( +POSTHOOK: query: CREATE EXTERNAL TABLE HBASE_TABLE_TEST_2( cvalue string , pk string , ccount int ) @@ -99,7 +102,8 @@ WITH SERDEPROPERTIES ( 'serialization.format'='1') TBLPROPERTIES ( 'hbase.table.name'='hbase_table_test_2', - 'serialization.null.format'='') + 'serialization.null.format'='', + 'external.table.purge' = 'true') POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@HBASE_TABLE_TEST_2 diff --git a/hbase-handler/src/test/results/positive/hbasestats.q.out b/hbase-handler/src/test/results/positive/hbasestats.q.out index e2061918dc..b6737b7d66 100644 --- a/hbase-handler/src/test/results/positive/hbasestats.q.out +++ b/hbase-handler/src/test/results/positive/hbasestats.q.out @@ -2,19 +2,21 @@ PREHOOK: query: DROP TABLE users PREHOOK: type: DROPTABLE POSTHOOK: query: DROP TABLE users POSTHOOK: type: DROPTABLE -PREHOOK: query: CREATE TABLE users(key string, state string, country string, country_id int) +PREHOOK: query: CREATE EXTERNAL TABLE users(key string, state string, country string, country_id int) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.columns.mapping" = "info:state,info:country,info:country_id" ) +TBLPROPERTIES ("external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@users -POSTHOOK: query: CREATE TABLE users(key string, state string, country string, country_id int) +POSTHOOK: query: CREATE EXTERNAL TABLE users(key string, state string, country string, country_id int) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ( "hbase.columns.mapping" = "info:state,info:country,info:country_id" ) +TBLPROPERTIES ("external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@users @@ -35,10 +37,12 @@ Database: default #### A masked pattern was here #### Retention: 0 #### A masked pattern was here #### -Table Type: MANAGED_TABLE +Table Type: EXTERNAL_TABLE Table Parameters: COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"country\":\"true\",\"country_id\":\"true\",\"key\":\"true\",\"state\":\"true\"}} + EXTERNAL TRUE bucketing_version 2 + external.table.purge true numFiles 0 numRows 0 rawDataSize 0 @@ -124,9 +128,11 @@ Database: default #### A masked pattern was here #### Retention: 0 #### A masked pattern was here #### -Table Type: MANAGED_TABLE +Table Type: EXTERNAL_TABLE Table Parameters: + EXTERNAL TRUE bucketing_version 2 + external.table.purge true #### A masked pattern was here #### numFiles 0 numRows 0 @@ -189,9 +195,11 @@ Database: default #### A masked pattern was here #### Retention: 0 #### A masked pattern was here #### -Table Type: MANAGED_TABLE +Table Type: EXTERNAL_TABLE Table Parameters: + EXTERNAL TRUE bucketing_version 2 + external.table.purge true #### A masked pattern was here #### numFiles 0 numRows 0 @@ -245,10 +253,12 @@ Database: default #### A masked pattern was here #### Retention: 0 #### A masked pattern was here #### -Table Type: MANAGED_TABLE +Table Type: EXTERNAL_TABLE Table Parameters: COLUMN_STATS_ACCURATE {\"BASIC_STATS\":\"true\"} + EXTERNAL TRUE bucketing_version 2 + external.table.purge true #### A masked pattern was here #### numFiles 0 numRows 2 @@ -349,9 +359,11 @@ Database: default #### A masked pattern was here #### Retention: 0 #### A masked pattern was here #### -Table Type: MANAGED_TABLE +Table Type: EXTERNAL_TABLE Table Parameters: + EXTERNAL TRUE bucketing_version 2 + external.table.purge true #### A masked pattern was here #### numFiles 0 numRows 2 diff --git a/hbase-handler/src/test/results/positive/ppd_key_ranges.q.out b/hbase-handler/src/test/results/positive/ppd_key_ranges.q.out index 18441e88ec..8112046715 100644 --- a/hbase-handler/src/test/results/positive/ppd_key_ranges.q.out +++ b/hbase-handler/src/test/results/positive/ppd_key_ranges.q.out @@ -1,12 +1,14 @@ -PREHOOK: query: CREATE TABLE hbase_ppd_keyrange(key int, value string) +PREHOOK: query: CREATE EXTERNAL TABLE hbase_ppd_keyrange(key int, value string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key#binary,cf:string") +TBLPROPERTIES ("external.table.purge" = "true") PREHOOK: type: CREATETABLE PREHOOK: Output: database:default PREHOOK: Output: default@hbase_ppd_keyrange -POSTHOOK: query: CREATE TABLE hbase_ppd_keyrange(key int, value string) +POSTHOOK: query: CREATE EXTERNAL TABLE hbase_ppd_keyrange(key int, value string) STORED BY 'org.apache.hadoop.hive.hbase.HBaseStorageHandler' WITH SERDEPROPERTIES ("hbase.columns.mapping" = ":key#binary,cf:string") +TBLPROPERTIES ("external.table.purge" = "true") POSTHOOK: type: CREATETABLE POSTHOOK: Output: database:default POSTHOOK: Output: default@hbase_ppd_keyrange