diff --git itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java index 1cf47c36cb490ce0b17ffe312cd2e9fc4bb7cd9a..d9fc1a927fc7a46e5239128254a464cd9141ca22 100644 --- itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java +++ itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java @@ -76,7 +76,6 @@ import org.apache.hadoop.hive.ql.Driver; import org.apache.hadoop.hive.ql.session.SessionState; import org.junit.Before; -import org.junit.BeforeClass; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -85,7 +84,7 @@ * Tests DbNotificationListener when used as a transactional event listener * (hive.metastore.transactional.event.listeners) */ -public class TestDbNotificationListener { +public abstract class TestDbNotificationListener { private static final Logger LOG = LoggerFactory.getLogger(TestDbNotificationListener.class .getName()); private static final int EVENTS_TTL = 30; @@ -94,15 +93,14 @@ private static IMetaStoreClient msClient; private static Driver driver; private static MessageDeserializer md = null; + private static boolean isTransactionalListener; private int startTime; private long firstEventId; - @SuppressWarnings("rawtypes") - @BeforeClass - public static void connectToMetastore() throws Exception { + public static void connectToMetastore(HiveConf.ConfVars listenerConfigName) throws Exception { HiveConf conf = new HiveConf(); - conf.setVar(HiveConf.ConfVars.METASTORE_TRANSACTIONAL_EVENT_LISTENERS, - DbNotificationListener.class.getName()); + conf.setVar(listenerConfigName, DbNotificationListener.class.getName()); + isTransactionalListener = listenerConfigName.equals(HiveConf.ConfVars.METASTORE_TRANSACTIONAL_EVENT_LISTENERS); conf.setVar(HiveConf.ConfVars.METASTORE_EVENT_DB_LISTENER_TTL, String.valueOf(EVENTS_TTL) + "s"); conf.setBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, false); conf.setBoolVar(HiveConf.ConfVars.FIRE_EVENTS_FOR_DML, true); @@ -166,16 +164,18 @@ public void createDatabase() throws Exception { // When hive.metastore.transactional.event.listeners is set, // a failed event should not create a new notification - DummyRawStoreFailEvent.setEventSucceed(false); - db = new Database(dbName2, dbDescription, dbLocationUri, emptyParameters); - try { - msClient.createDatabase(db); - fail("Error: create database should've failed"); - } catch (Exception ex) { - // expected + if (isTransactionalListener) { + DummyRawStoreFailEvent.setEventSucceed(false); + db = new Database(dbName2, dbDescription, dbLocationUri, emptyParameters); + try { + msClient.createDatabase(db); + fail("Error: create database should've failed"); + } catch (Exception ex) { + // expected + } + rsp = msClient.getNextNotification(firstEventId, 0, null); + assertEquals(1, rsp.getEventsSize()); } - rsp = msClient.getNextNotification(firstEventId, 0, null); - assertEquals(1, rsp.getEventsSize()); } @Test @@ -208,17 +208,19 @@ public void dropDatabase() throws Exception { // When hive.metastore.transactional.event.listeners is set, // a failed event should not create a new notification - db = new Database(dbName2, dbDescription, dbLocationUri, emptyParameters); - msClient.createDatabase(db); - DummyRawStoreFailEvent.setEventSucceed(false); - try { - msClient.dropDatabase(dbName2); - fail("Error: drop database should've failed"); - } catch (Exception ex) { - // expected + if (isTransactionalListener) { + db = new Database(dbName2, dbDescription, dbLocationUri, emptyParameters); + msClient.createDatabase(db); + DummyRawStoreFailEvent.setEventSucceed(false); + try { + msClient.dropDatabase(dbName2); + fail("Error: drop database should've failed"); + } catch (Exception ex) { + // expected + } + rsp = msClient.getNextNotification(firstEventId, 0, null); + assertEquals(3, rsp.getEventsSize()); } - rsp = msClient.getNextNotification(firstEventId, 0, null); - assertEquals(3, rsp.getEventsSize()); } @Test @@ -258,18 +260,20 @@ public void createTable() throws Exception { // When hive.metastore.transactional.event.listeners is set, // a failed event should not create a new notification - table = - new Table(tblName2, defaultDbName, tblOwner, startTime, startTime, 0, sd, null, - emptyParameters, null, null, null); - DummyRawStoreFailEvent.setEventSucceed(false); - try { - msClient.createTable(table); - fail("Error: create table should've failed"); - } catch (Exception ex) { - // expected + if (isTransactionalListener) { + table = + new Table(tblName2, defaultDbName, tblOwner, startTime, startTime, 0, sd, null, + emptyParameters, null, null, null); + DummyRawStoreFailEvent.setEventSucceed(false); + try { + msClient.createTable(table); + fail("Error: create table should've failed"); + } catch (Exception ex) { + // expected + } + rsp = msClient.getNextNotification(firstEventId, 0, null); + assertEquals(1, rsp.getEventsSize()); } - rsp = msClient.getNextNotification(firstEventId, 0, null); - assertEquals(1, rsp.getEventsSize()); } @Test @@ -314,15 +318,17 @@ public void alterTable() throws Exception { // When hive.metastore.transactional.event.listeners is set, // a failed event should not create a new notification - DummyRawStoreFailEvent.setEventSucceed(false); - try { - msClient.alter_table(defaultDbName, tblName, table); - fail("Error: alter table should've failed"); - } catch (Exception ex) { - // expected + if (isTransactionalListener) { + DummyRawStoreFailEvent.setEventSucceed(false); + try { + msClient.alter_table(defaultDbName, tblName, table); + fail("Error: alter table should've failed"); + } catch (Exception ex) { + // expected + } + rsp = msClient.getNextNotification(firstEventId, 0, null); + assertEquals(2, rsp.getEventsSize()); } - rsp = msClient.getNextNotification(firstEventId, 0, null); - assertEquals(2, rsp.getEventsSize()); } @Test @@ -365,19 +371,21 @@ public void dropTable() throws Exception { // When hive.metastore.transactional.event.listeners is set, // a failed event should not create a new notification - table = - new Table(tblName2, defaultDbName, tblOwner, startTime, startTime, 0, sd, null, - emptyParameters, null, null, null); - msClient.createTable(table); - DummyRawStoreFailEvent.setEventSucceed(false); - try { - msClient.dropTable(defaultDbName, tblName2); - fail("Error: drop table should've failed"); - } catch (Exception ex) { - // expected + if (isTransactionalListener) { + table = + new Table(tblName2, defaultDbName, tblOwner, startTime, startTime, 0, sd, null, + emptyParameters, null, null, null); + msClient.createTable(table); + DummyRawStoreFailEvent.setEventSucceed(false); + try { + msClient.dropTable(defaultDbName, tblName2); + fail("Error: drop table should've failed"); + } catch (Exception ex) { + // expected + } + rsp = msClient.getNextNotification(firstEventId, 0, null); + assertEquals(3, rsp.getEventsSize()); } - rsp = msClient.getNextNotification(firstEventId, 0, null); - assertEquals(3, rsp.getEventsSize()); } @Test @@ -430,18 +438,20 @@ public void addPartition() throws Exception { // When hive.metastore.transactional.event.listeners is set, // a failed event should not create a new notification - partition = - new Partition(Arrays.asList("tomorrow"), defaultDbName, tblName2, startTime, startTime, sd, - emptyParameters); - DummyRawStoreFailEvent.setEventSucceed(false); - try { - msClient.add_partition(partition); - fail("Error: add partition should've failed"); - } catch (Exception ex) { - // expected + if (isTransactionalListener) { + partition = + new Partition(Arrays.asList("tomorrow"), defaultDbName, tblName2, startTime, startTime, sd, + emptyParameters); + DummyRawStoreFailEvent.setEventSucceed(false); + try { + msClient.add_partition(partition); + fail("Error: add partition should've failed"); + } catch (Exception ex) { + // expected + } + rsp = msClient.getNextNotification(firstEventId, 0, null); + assertEquals(2, rsp.getEventsSize()); } - rsp = msClient.getNextNotification(firstEventId, 0, null); - assertEquals(2, rsp.getEventsSize()); } @Test @@ -496,15 +506,17 @@ public void alterPartition() throws Exception { // When hive.metastore.transactional.event.listeners is set, // a failed event should not create a new notification - DummyRawStoreFailEvent.setEventSucceed(false); - try { - msClient.alter_partition(defaultDbName, tblName, newPart, null); - fail("Error: alter partition should've failed"); - } catch (Exception ex) { - // expected + if (isTransactionalListener) { + DummyRawStoreFailEvent.setEventSucceed(false); + try { + msClient.alter_partition(defaultDbName, tblName, newPart, null); + fail("Error: alter partition should've failed"); + } catch (Exception ex) { + // expected + } + rsp = msClient.getNextNotification(firstEventId, 0, null); + assertEquals(3, rsp.getEventsSize()); } - rsp = msClient.getNextNotification(firstEventId, 0, null); - assertEquals(3, rsp.getEventsSize()); } @Test @@ -559,24 +571,31 @@ public void dropPartition() throws Exception { // When hive.metastore.transactional.event.listeners is set, // a failed event should not create a new notification - List newpartCol1Vals = Arrays.asList("tomorrow"); - partition = - new Partition(newpartCol1Vals, defaultDbName, tblName, startTime, startTime, sd, - emptyParameters); - msClient.add_partition(partition); - DummyRawStoreFailEvent.setEventSucceed(false); - try { - msClient.dropPartition(defaultDbName, tblName, newpartCol1Vals, false); - fail("Error: drop partition should've failed"); - } catch (Exception ex) { - // expected + if (isTransactionalListener) { + List newpartCol1Vals = Arrays.asList("tomorrow"); + partition = + new Partition(newpartCol1Vals, defaultDbName, tblName, startTime, startTime, sd, + emptyParameters); + msClient.add_partition(partition); + DummyRawStoreFailEvent.setEventSucceed(false); + try { + msClient.dropPartition(defaultDbName, tblName, newpartCol1Vals, false); + fail("Error: drop partition should've failed"); + } catch (Exception ex) { + // expected + } + rsp = msClient.getNextNotification(firstEventId, 0, null); + assertEquals(4, rsp.getEventsSize()); } - rsp = msClient.getNextNotification(firstEventId, 0, null); - assertEquals(4, rsp.getEventsSize()); } @Test public void exchangePartition() throws Exception { + // TODO: No support for non-transactional listeners for exchange partitions currently + if (!isTransactionalListener) { + return; + } + String dbName = "default"; List cols = new ArrayList(); cols.add(new FieldSchema("col1", "int", "nocomment")); @@ -695,19 +714,21 @@ public void createFunction() throws Exception { // When hive.metastore.transactional.event.listeners is set, // a failed event should not create a new notification - DummyRawStoreFailEvent.setEventSucceed(false); - func = - new Function(funcName2, defaultDbName, funcClass2, ownerName, PrincipalType.USER, - startTime, FunctionType.JAVA, Arrays.asList(new ResourceUri(ResourceType.JAR, - funcResource2))); - try { - msClient.createFunction(func); - fail("Error: create function should've failed"); - } catch (Exception ex) { - // expected + if (isTransactionalListener) { + DummyRawStoreFailEvent.setEventSucceed(false); + func = + new Function(funcName2, defaultDbName, funcClass2, ownerName, PrincipalType.USER, + startTime, FunctionType.JAVA, Arrays.asList(new ResourceUri(ResourceType.JAR, + funcResource2))); + try { + msClient.createFunction(func); + fail("Error: create function should've failed"); + } catch (Exception ex) { + // expected + } + rsp = msClient.getNextNotification(firstEventId, 0, null); + assertEquals(1, rsp.getEventsSize()); } - rsp = msClient.getNextNotification(firstEventId, 0, null); - assertEquals(1, rsp.getEventsSize()); } @Test @@ -744,20 +765,22 @@ public void dropFunction() throws Exception { // When hive.metastore.transactional.event.listeners is set, // a failed event should not create a new notification - func = - new Function(funcName2, defaultDbName, funcClass2, ownerName, PrincipalType.USER, - startTime, FunctionType.JAVA, Arrays.asList(new ResourceUri(ResourceType.JAR, - funcResource2))); - msClient.createFunction(func); - DummyRawStoreFailEvent.setEventSucceed(false); - try { - msClient.dropFunction(defaultDbName, funcName2); - fail("Error: drop function should've failed"); - } catch (Exception ex) { - // expected + if (isTransactionalListener) { + func = + new Function(funcName2, defaultDbName, funcClass2, ownerName, PrincipalType.USER, + startTime, FunctionType.JAVA, Arrays.asList(new ResourceUri(ResourceType.JAR, + funcResource2))); + msClient.createFunction(func); + DummyRawStoreFailEvent.setEventSucceed(false); + try { + msClient.dropFunction(defaultDbName, funcName2); + fail("Error: drop function should've failed"); + } catch (Exception ex) { + // expected + } + rsp = msClient.getNextNotification(firstEventId, 0, null); + assertEquals(3, rsp.getEventsSize()); } - rsp = msClient.getNextNotification(firstEventId, 0, null); - assertEquals(3, rsp.getEventsSize()); } @Test @@ -809,21 +832,23 @@ public void createIndex() throws Exception { // When hive.metastore.transactional.event.listeners is set, // a failed event should not create a new notification - DummyRawStoreFailEvent.setEventSucceed(false); - index = - new Index("createIndexTable2", null, "default", tableName, startTime, startTime, - "createIndexTable2__createIndexTable2__", sd, emptyParameters, false); - Table indexTable2 = - new Table("createIndexTable2__createIndexTable2__", dbName, "me", startTime, startTime, 0, - sd, null, emptyParameters, null, null, null); - try { - msClient.createIndex(index, indexTable2); - fail("Error: create index should've failed"); - } catch (Exception ex) { - // expected + if (isTransactionalListener) { + DummyRawStoreFailEvent.setEventSucceed(false); + index = + new Index("createIndexTable2", null, "default", tableName, startTime, startTime, + "createIndexTable2__createIndexTable2__", sd, emptyParameters, false); + Table indexTable2 = + new Table("createIndexTable2__createIndexTable2__", dbName, "me", startTime, startTime, 0, + sd, null, emptyParameters, null, null, null); + try { + msClient.createIndex(index, indexTable2); + fail("Error: create index should've failed"); + } catch (Exception ex) { + // expected + } + rsp = msClient.getNextNotification(firstEventId, 0, null); + assertEquals(3, rsp.getEventsSize()); } - rsp = msClient.getNextNotification(firstEventId, 0, null); - assertEquals(3, rsp.getEventsSize()); } @Test @@ -875,24 +900,26 @@ public void dropIndex() throws Exception { // When hive.metastore.transactional.event.listeners is set, // a failed event should not create a new notification - index = - new Index("dropIndexTable2", null, "default", tableName, startTime, startTime, - "dropIndexTable__dropIndexTable2__", sd, emptyParameters, false); - Table indexTable2 = - new Table("dropIndexTable__dropIndexTable2__", dbName, "me", startTime, startTime, 0, sd, - null, emptyParameters, null, null, null); - msClient.createIndex(index, indexTable2); - DummyRawStoreFailEvent.setEventSucceed(false); - try { - // drops index and indexTable - msClient.dropIndex(dbName, tableName, "dropIndex2", true); - fail("Error: drop index should've failed"); - } catch (Exception ex) { - // expected - } + if (isTransactionalListener) { + index = + new Index("dropIndexTable2", null, "default", tableName, startTime, startTime, + "dropIndexTable__dropIndexTable2__", sd, emptyParameters, false); + Table indexTable2 = + new Table("dropIndexTable__dropIndexTable2__", dbName, "me", startTime, startTime, 0, sd, + null, emptyParameters, null, null, null); + msClient.createIndex(index, indexTable2); + DummyRawStoreFailEvent.setEventSucceed(false); + try { + // drops index and indexTable + msClient.dropIndex(dbName, tableName, "dropIndex2", true); + fail("Error: drop index should've failed"); + } catch (Exception ex) { + // expected + } - rsp = msClient.getNextNotification(firstEventId, 0, null); - assertEquals(6, rsp.getEventsSize()); + rsp = msClient.getNextNotification(firstEventId, 0, null); + assertEquals(6, rsp.getEventsSize()); + } } @Test @@ -949,15 +976,17 @@ public void alterIndex() throws Exception { // When hive.metastore.transactional.event.listeners is set, // a failed event should not create a new notification - DummyRawStoreFailEvent.setEventSucceed(false); - try { - msClient.alter_index(dbName, tableName, indexName, newIndex); - fail("Error: alter index should've failed"); - } catch (Exception ex) { - // expected + if (isTransactionalListener) { + DummyRawStoreFailEvent.setEventSucceed(false); + try { + msClient.alter_index(dbName, tableName, indexName, newIndex); + fail("Error: alter index should've failed"); + } catch (Exception ex) { + // expected + } + rsp = msClient.getNextNotification(firstEventId, 0, null); + assertEquals(4, rsp.getEventsSize()); } - rsp = msClient.getNextNotification(firstEventId, 0, null); - assertEquals(4, rsp.getEventsSize()); } @Test @@ -1244,10 +1273,11 @@ public void sqlInsertPartition() throws Exception { // Event 22, 23, 24 driver.run("insert overwrite table " + tblName + " partition(ds='todaytwo') select c from " + tblName + " where 'ds'='today'"); + driver.run("drop table " + tblName); // Get notifications from metastore NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null); - assertEquals(24, rsp.getEventsSize()); + assertEquals(25, rsp.getEventsSize()); NotificationEvent event = rsp.getEvents().get(1); assertEquals(firstEventId + 2, event.getEventId()); assertEquals(EventType.ADD_PARTITION.toString(), event.getEventType()); diff --git itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestNonTransactionalDbNotificationListener.java itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestNonTransactionalDbNotificationListener.java new file mode 100644 index 0000000000000000000000000000000000000000..a4a0bcfb8f3378b29d1ec7229068fb98ddb6b677 --- /dev/null +++ itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestNonTransactionalDbNotificationListener.java @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.hive.hcatalog.listener; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.junit.BeforeClass; + +public class TestNonTransactionalDbNotificationListener extends TestDbNotificationListener { + + @SuppressWarnings("rawtypes") + @BeforeClass + public static void setListenerAndConnectToMetastore() throws Exception { + connectToMetastore(HiveConf.ConfVars.METASTORE_EVENT_LISTENERS); + } +} diff --git itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestTransactionalDbNotificationListener.java itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestTransactionalDbNotificationListener.java new file mode 100644 index 0000000000000000000000000000000000000000..b661ce177fda15d9482eaff37f4539b5094b0c53 --- /dev/null +++ itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestTransactionalDbNotificationListener.java @@ -0,0 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.hive.hcatalog.listener; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.junit.BeforeClass; + +public class TestTransactionalDbNotificationListener extends TestDbNotificationListener { + + @SuppressWarnings("rawtypes") + @BeforeClass + public static void setListenerAndConnectToMetastore() throws Exception { + connectToMetastore(HiveConf.ConfVars.METASTORE_TRANSACTIONAL_EVENT_LISTENERS); + } +}