Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.
Comment: Migrated to Confluence 5.3

Overview

Notification mechanism is introduced in Hcatalog so that a user can be notified of certain events occurring in Metastore if she desires so. Events are of six types: add_database, add_table, add_partition, drop_partition, drop_table, drop_database. When any of these events occur on Metastore message is sent to a Message Bus. Any user who wish to get notification for these messages can subscribe to a message bus. Once subscribed, message bus will deliver the messages to the subscriber.

Topic Name

The client in order to receive message must be subscribed to the right topic. Topic name for different types of events are different and is configurable in few cases. All default topic names are prefixed with a string which is configurable in hive-site.xml. The configuration key is hcat.msgbus.topic.prefix which defaults to hcat. We shall refer to the value of the topic prefix as HCAT_TOPIC_PREFIX in rest of the document. For example drop_database events would be sent to topic HCAT_TOPIC_PREFIX (e.g. hcat). add_partition messages would be sent to HCAT_TOPIC_PREFIX.DB_NAME.TABLE_NAME (eg: hcat.default.example_table). But the topic name for events on a table could be changed per table by user. For example following hive query sets the topic name for the table example_table to the string example_table_topic_name:

Code Block

ALTER TABLE example_table SET TBLPROPERTIES ("hcat.msgbus.topic.name" = "example_table_topic_name")
How to receive notification

To start receiving message you first need to create a connection to messagebus as demonstrated below:

...

Then subscribe to a topic you are interested in. While subscribing on a message bus, user need to subscribe to a particular topic to receive the messages which are being delivered on that topic. The topic name corresponding to a particular table is stored in table properties and can be retrieved using following piece of code:Following example shows how the subscriber first finds out the topic name for the table she is interested in, and then subscribes to it.

Code Block
    HiveMetaStoreClient msc = new HiveMetaStoreClient(hiveConf);
    String topicName = msc.getTable("mydb", "myTbl").getParameters().get(HCatConstants.HCAT_MSGBUS_TOPIC_NAME);

...

Event type-string: "CREATE_DATABASE"
Topic Name: HCAT_TOPIC_PREFIX
Example JSON Format:

Code Block
{
  "timestamp" : 1360272556,
  "eventType" : "CREATE_DATABASE",
  "server"    : "hcatserver.mydomain.net",
  "servicePrincipal" : "hcat/hcatserver@MYDOMAIN.NET",
  "db"        : "mydb"
}

...

Event type-string: "DROP_DATABASE"
Topic Name: HCAT_TOPIC_PREFIX
Example JSON Format:

Code Block
{
  "timestamp" : 1360272556,
  "eventType" : "DROP_DATABASE",
  "server"    : "hcatserver.mydomain.net",
  "servicePrincipal" : "hcat/hcatserver@MYDOMAIN.NET",
  "db"        : "mydb"
}

...

Event type-string: "CREATE_TABLE"
Topic Name: HCAT_TOPIC_PREFIX.DB_NAME
Example JSON Format:

Code Block
{
  "timestamp" : 1360272556,
  "eventType" : "CREATE_TABLE",
  "server"    : "hcatserver.mydomain.net",
  "servicePrincipal" : "hcat/hcatserver@MYDOMAIN.NET",
  "db"        : "mydb",
  "table"     : "mytbl" 
}

...

Event type-string: "DROP_TABLE"
Topic Name: HCAT_TOPIC_PREFIX.DB_NAME
Example JSON Format:

Code Block
{
  "timestamp" : 1360272556,
  "eventType" : "DROP_TABLE",
  "server"    : "hcatserver.mydomain.net",
  "servicePrincipal" : "hcat/hcatserver@MYDOMAIN.NET",
  "db"        : "mydb",
  "table"     : "mytbl" 
}

...

Event type-string: "ADD_PARTITION"
Topic Name: HCAT_TOPIC_PREFIX.DB_NAME.TABLE_NAME (default) but is user configurable
Example JSON Format:

Code Block
{
  "timestamp" : 1360272556,
  "eventType" : "ADD_PARTITION",
  "server"    : "hcatserver.mydomain.net",
  "servicePrincipal" : "hcat/hcatserver@MYDOMAIN.NET",
  "db"        : "mydb",
  "table"     : "mytbl",
  "partitions": [
                   { "partKey1" : "partVal1A", "partKey2" : "partVal2A" },
                   { "partKey1" : "partVal1B", "partKey2" : "partVal2B" },
                   { "partKey1" : "partVal1C", "partKey2" : "partVal2C" }
                ]
}

...

Event type-string: "DROP_PARTITION"
Topic Name: HCAT_TOPIC_PREFIX.DB_NAME.TABLE_NAME (default) but is user configurable
Example JSON Format:

Code Block
{
  "timestamp" : 1360272556,
  "eventType" : "DROP_PARTITION",
  "server"    : "hcatserver.mydomain.net",
  "servicePrincipal" : "hcat/hcatserver@MYDOMAIN.NET",
  "db"        : "mydb",
  "table"     : "mytbl",
  "partitions": [
                   { "partKey1" : "partVal1A", "partKey2" : "partVal2A" },
                   { "partKey1" : "partVal1B", "partKey2" : "partVal2B" },
                   { "partKey1" : "partVal1C", "partKey2" : "partVal2C" }
                ]
}

...