IMPALA-10502: Handle CREATE/DROP events correctly

The current way to detect self-events in case of CREATE/DROP events on
database, table and partition is problematic when the same object is
created and dropped repeatedly in quick succession. This happens mainly
due to couple of reasons. For example if we have the below
sequence of DDLs in Impala:
1. create table foo; --> catalogd creates table foo
2. drop table foo; --> catalogd drops table foo
...
Events processor receives the CREATE_TABLE event pertaining to (1)
above. Now it cannot determine whether the table needs to be created
or not. Similarly, if we interchange the order of DROP and CREATE
statements above, the DROP_TABLE event received by the events processor
will unnecessarily remove the table when it should not.

This can cause problems for queries which expect the table to exist or
not exist. E.g create table query fails with a table already exists or
a drop table query fails with table does not exist error.

In order to fix this issue, catalogd now keeps track of dropped objects
in a deleteLog which are garbage collected as the events come in. Every
time a database, table or partition is dropped, the deleteLog is
populated with the drop event id generated due to the drop
operation. This deleteLog is looked up when the event is received to
determine if the event can be ignored. Additionally, catalogd keeps
track of the create event id at the Database, Table or Partition level
during the create DDL execution so that the event can be ignored later
by events processor.

Testing:
1. Added test_create_drop_events and test_local_catalog_create_drop_events
test which loops to create create/drop events for database, table and
partitions.
2. Added new metrics which the test verifies to ensure that events
don't create or drop the object.

Change-Id: Ia2c5e96b48abac015240f20295b3ec3b1d71f24a
Reviewed-on: http://gerrit.cloudera.org:8080/17308
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Reviewed-by: Vihang Karajgaonkar <vihang@cloudera.com>
This commit is contained in:
Vihang Karajgaonkar
2021-04-12 14:19:17 -07:00
committed by Vihang Karajgaonkar
parent 00c8e157dd
commit 7f7a631e92
29 changed files with 2648 additions and 1180 deletions

View File

@@ -39,6 +39,7 @@ import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.stream.Collectors;
import javax.annotation.Nullable;
import org.apache.commons.collections.MapUtils;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.hdfs.DistributedFileSystem;
@@ -212,7 +213,7 @@ import com.google.common.collect.Maps;
public class CatalogServiceCatalog extends Catalog {
public static final Logger LOG = LoggerFactory.getLogger(CatalogServiceCatalog.class);
private static final int INITIAL_META_STORE_CLIENT_POOL_SIZE = 10;
public static final int INITIAL_META_STORE_CLIENT_POOL_SIZE = 10;
private static final int MAX_NUM_SKIPPED_TOPIC_UPDATES = 2;
private final int maxSkippedUpdatesLockContention_;
//value of timeout for the topic update thread while waiting on the table lock.
@@ -280,7 +281,7 @@ public class CatalogServiceCatalog extends Catalog {
// Manages the event processing from metastore for issuing invalidates on tables
private ExternalEventsProcessor metastoreEventProcessor_;
private final ICatalogMetastoreServer catalogMetastoreServer_;
private ICatalogMetastoreServer catalogMetastoreServer_;
/**
* See the gflag definition in be/.../catalog-server.cc for details on these modes.
@@ -354,12 +355,13 @@ public class CatalogServiceCatalog extends Catalog {
BackendConfig.INSTANCE.getBackendCfg().catalog_topic_mode.toUpperCase());
catalogdTableInvalidator_ = CatalogdTableInvalidator.create(this,
BackendConfig.INSTANCE);
metastoreEventProcessor_ = getEventsProcessor();
Preconditions.checkState(PARTIAL_FETCH_RPC_QUEUE_TIMEOUT_S > 0);
// start polling for metastore events
}
public void startEventsProcessor() {
Preconditions.checkNotNull(metastoreEventProcessor_,
"Start events processor called before initializing it");
metastoreEventProcessor_.start();
catalogMetastoreServer_ = getCatalogMetastoreServer();
catalogMetastoreServer_.start();
}
/**
@@ -375,20 +377,6 @@ public class CatalogServiceCatalog extends Catalog {
initialHmsCnxnTimeoutSec));
}
/**
* Returns an instance of CatalogMetastoreServer if start_hms_server configuration is
* true. Otherwise, returns a NoOpCatalogMetastoreServer
*/
@VisibleForTesting
protected ICatalogMetastoreServer getCatalogMetastoreServer() {
if (!BackendConfig.INSTANCE.startHmsServer()) {
return NoOpCatalogMetastoreServer.INSTANCE;
}
int portNumber = BackendConfig.INSTANCE.getHMSPort();
Preconditions.checkState(portNumber > 0, "Invalid port number for HMS service.");
return new CatalogMetastoreServer(this);
}
/**
* Check whether the database is in blacklist
*/
@@ -416,36 +404,6 @@ public class CatalogServiceCatalog extends Catalog {
authzManager_ = Preconditions.checkNotNull(authzManager);
}
/**
* Returns a Metastore event processor object if
* <code>BackendConfig#getHMSPollingIntervalInSeconds</code> returns a non-zero
*.value of polling interval. Otherwise, returns a no-op events processor. It is
* important to fetch the current notification event id at the Catalog service
* initialization time so that event processor starts to sync at the event id
* corresponding to the catalog start time.
*/
private ExternalEventsProcessor getEventsProcessor() throws ImpalaException {
long eventPollingInterval = BackendConfig.INSTANCE.getHMSPollingIntervalInSeconds();
if (eventPollingInterval <= 0) {
LOG.info(String
.format("Metastore event processing is disabled. Event polling interval is %d",
eventPollingInterval));
return NoOpEventProcessor.getInstance();
}
try (MetaStoreClient metaStoreClient = getMetaStoreClient()) {
CurrentNotificationEventId currentNotificationId =
metaStoreClient.getHiveClient().getCurrentNotificationEventId();
return MetastoreEventsProcessor.getInstance(
this, currentNotificationId.getEventId(), eventPollingInterval);
} catch (TException e) {
LOG.error("Unable to fetch the current notification event id from metastore."
+ "Metastore event processing will be disabled.", e);
throw new CatalogException(
"Fatal error while initializing metastore event processor", e);
}
}
@VisibleForTesting
public ExternalEventsProcessor getMetastoreEventProcessor() {
return metastoreEventProcessor_;
}
@@ -989,7 +947,7 @@ public class CatalogServiceCatalog extends Catalog {
// no version info or service id in the event
if (versionNumber == -1 || serviceIdFromEvent.isEmpty()) {
LOG.info("Not a self-event since the given version is {} and service id is {}",
versionNumber, serviceIdFromEvent);
versionNumber, serviceIdFromEvent.isEmpty() ? "empty" : serviceIdFromEvent);
return false;
}
// if the service id from event doesn't match with our service id this is not a
@@ -1912,11 +1870,11 @@ public class CatalogServiceCatalog extends Catalog {
// the event ids, if there is external DDL activity on metastore during reset.
// Unfortunately, there is no good way to avoid this since HMS does not provide
// APIs which can fetch all the tables/databases at a given id. It is OKAY to
// re-process some of these events since event processor relies on creationTime of
// the objects to uniquely identify tables from create and drop events. In case of
// re-process some of these events since event processor relies on creation eventId
// to uniquely determine if the table was created/dropped by catalogd. In case of
// alter events, however it is likely that some tables would be unnecessarily
// invalidated. That would happen when during reset, there were external alter events
// and by the time we processed them, Catalog had already loaded them.
// refreshed. That would happen when during reset, there were external alter events
// and by the time we processed them, catalog had already loaded them.
long currentEventId = metastoreEventProcessor_.getCurrentEventId();
// pause the event processing since the cache is anyways being cleared
metastoreEventProcessor_.pause();
@@ -2003,15 +1961,21 @@ public class CatalogServiceCatalog extends Catalog {
return startVersion;
}
public Db addDb(String dbName, org.apache.hadoop.hive.metastore.api.Database msDb) {
return addDb(dbName, msDb, -1);
}
/**
* Adds a database name to the metadata cache and returns the database's
* new Db object. Used by CREATE DATABASE statements.
*/
public Db addDb(String dbName, org.apache.hadoop.hive.metastore.api.Database msDb) {
public Db addDb(String dbName, org.apache.hadoop.hive.metastore.api.Database msDb,
long eventId) {
Db newDb = new Db(dbName, msDb);
versionLock_.writeLock().lock();
try {
newDb.setCatalogVersion(incrementAndGetCatalogVersion());
newDb.setCreateEventId(eventId);
addDb(newDb);
return newDb;
} finally {
@@ -2019,25 +1983,6 @@ public class CatalogServiceCatalog extends Catalog {
}
}
/**
* Adds a database name to the metadata cache if not exists and returns the
* true is a new Db Object was added. Used by MetastoreEventProcessor to handle
* CREATE_DATABASE events
*/
public boolean addDbIfNotExists(
String dbName, org.apache.hadoop.hive.metastore.api.Database msDb) {
versionLock_.writeLock().lock();
try {
Db db = getDb(dbName);
if (db == null) {
return addDb(dbName, msDb) != null;
}
return false;
} finally {
versionLock_.writeLock().unlock();
}
}
/**
* Removes a database from the metadata cache and returns the removed database,
* or null if the database did not exist in the cache.
@@ -2055,39 +2000,6 @@ public class CatalogServiceCatalog extends Catalog {
}
}
/**
* @param msDb Metastore Database used to remove Db from Catalog
* @param dbFound Set to true if Database is found in Catalog
* @param dbMatched Set to true if Database is found in Catalog and it's CREATION_TIME
* is equal to the metastore DB
* @return the DB object removed. Return null if DB does not exist or was not removed
* because CREATION_TIME does not match.
*/
public Db removeDbIfExists(org.apache.hadoop.hive.metastore.api.Database msDb,
Reference<Boolean> dbFound, Reference<Boolean> dbMatched) {
dbFound.setRef(false);
dbMatched.setRef(false);
versionLock_.writeLock().lock();
try {
String dbName = msDb.getName();
Db catalogDb = getDb(dbName);
if (catalogDb == null) return null;
dbFound.setRef(true);
// Remove the DB only if the CREATION_TIME matches with the metastore DB from event.
if (msDb.getCreateTime() == catalogDb.getMetaStoreDb().getCreateTime()) {
Db removedDb = removeDb(dbName);
if (removedDb != null) {
dbMatched.setRef(true);
return removedDb;
}
}
return null;
} finally {
versionLock_.writeLock().unlock();
}
}
/**
* Helper function to clean up the state associated with a removed database. It creates
* the entries in the delete log for 'db' as well as for its tables and functions
@@ -2110,35 +2022,14 @@ public class CatalogServiceCatalog extends Catalog {
deleteLog_.addRemovedObject(db.toTCatalogObject());
}
/**
* Adds table with the given db and table name to the catalog if it does not exists.
* @return true if the table was successfully added and false if the table already
* exists
* @throws CatalogException if the db is not found
*/
public boolean addTableIfNotExists(String dbName, String tblName)
throws CatalogException {
versionLock_.writeLock().lock();
try {
Db db = getDb(dbName);
if (db == null) {
throw new CatalogException(String.format("Db %s does not exist", dbName));
}
Table existingTable = db.getTable(tblName);
if (existingTable != null) return false;
Table incompleteTable = IncompleteTable.createUninitializedTable(db, tblName);
incompleteTable.setCatalogVersion(incrementAndGetCatalogVersion());
db.addTable(incompleteTable);
return true;
} finally {
versionLock_.writeLock().unlock();
}
public Table addIncompleteTable(String dbName, String tblName) {
return addIncompleteTable(dbName, tblName, -1L);
}
/**
* Adds a table with the given name to the catalog and returns the new table.
*/
public Table addIncompleteTable(String dbName, String tblName) {
public Table addIncompleteTable(String dbName, String tblName, long createEventId) {
versionLock_.writeLock().lock();
try {
// IMPALA-9211: get db object after holding the writeLock in case of getting stale
@@ -2154,6 +2045,7 @@ public class CatalogServiceCatalog extends Catalog {
}
Table incompleteTable = IncompleteTable.createUninitializedTable(db, tblName);
incompleteTable.setCatalogVersion(incrementAndGetCatalogVersion());
incompleteTable.setCreateEventId(createEventId);
db.addTable(incompleteTable);
return db.getTable(tblName);
} finally {
@@ -2237,7 +2129,7 @@ public class CatalogServiceCatalog extends Catalog {
.inc();
}
previousCatalogVersion = tbl.getCatalogVersion();
loadReq = tableLoadingMgr_.loadAsync(tableName, reason);
loadReq = tableLoadingMgr_.loadAsync(tableName, tbl.getCreateEventId(), reason);
} finally {
versionLock_.readLock().unlock();
}
@@ -2290,50 +2182,6 @@ public class CatalogServiceCatalog extends Catalog {
}
}
/**
* Remove a catalog table based on the given metastore table if it exists and its
* id matches with the id of the table in Catalog.
*
* @param msTable Metastore table to be used to remove Table
* @param tblWasfound is set to true if the table was found in the catalog
* @param tblMatched is set to true if the table is found and it matched with the
* id of the cached metastore table in catalog or if the existing table is a
* incomplete table
* @return Removed table object. Return null if the table was not removed
*/
public Table removeTableIfExists(org.apache.hadoop.hive.metastore.api.Table msTable,
Reference<Boolean> tblWasfound, Reference<Boolean> tblMatched) {
tblWasfound.setRef(false);
tblMatched.setRef(false);
// make sure that the createTime of the input table is valid
Preconditions.checkState(msTable.getId() > 0);
versionLock_.writeLock().lock();
try {
Db db = getDb(msTable.getDbName());
if (db == null) return null;
Table tblToBeRemoved = db.getTable(msTable.getTableName());
if (tblToBeRemoved == null) return null;
tblWasfound.setRef(true);
// make sure that you are removing the same instance of the table object which
// is given by comparing the metastore createTime. In case the found table is a
// Incomplete table remove it
if (tblToBeRemoved instanceof IncompleteTable
|| (msTable.getId()
== tblToBeRemoved.getMetaStoreTable().getId())) {
tblMatched.setRef(true);
Table removedTbl = db.removeTable(tblToBeRemoved.getName());
removedTbl.setCatalogVersion(incrementAndGetCatalogVersion());
deleteLog_.addRemovedObject(removedTbl.toMinimalTCatalogObject());
return removedTbl;
}
return null;
} finally {
versionLock_.writeLock().unlock();
}
}
/**
* Removes a table from the catalog and increments the catalog version.
* Returns the removed Table, or null if the table or db does not exist.
@@ -2449,42 +2297,8 @@ public class CatalogServiceCatalog extends Catalog {
removeTable(oldTableName.getDb_name(), oldTableName.getTable_name());
if (oldTable == null) return Pair.create(null, null);
return Pair.create(oldTable,
addIncompleteTable(newTableName.getDb_name(), newTableName.getTable_name()));
} finally {
versionLock_.writeLock().unlock();
}
}
/**
* Renames the table by atomically removing oldTable and adding the newTable.
* @return true if oldTable was removed and newTable was added, false if oldTable or
* either of oldDb or newDb is not in catalog.
*/
public boolean renameTableIfExists(TTableName oldTableName,
TTableName newTableName) {
boolean tableRenamed = false;
versionLock_.writeLock().lock();
try {
Db oldDb = getDb(oldTableName.db_name);
Db newDb = getDb(newTableName.db_name);
if (oldDb != null && newDb != null) {
Table existingTable = removeTable(oldTableName.db_name, oldTableName.table_name);
// Add the newTable only if oldTable existed.
if (existingTable != null) {
if (existingTable instanceof HdfsTable) {
// Add the old instance to the deleteLog_ so we can send isDeleted updates for
// its partitions.
existingTable.setCatalogVersion(incrementAndGetCatalogVersion());
deleteLog_.addRemovedObject(existingTable.toMinimalTCatalogObject());
}
Table incompleteTable = IncompleteTable.createUninitializedTable(newDb,
newTableName.getTable_name());
incompleteTable.setCatalogVersion(incrementAndGetCatalogVersion());
newDb.addTable(incompleteTable);
tableRenamed = true;
}
}
return tableRenamed;
addIncompleteTable(newTableName.getDb_name(), newTableName.getTable_name(),
oldTable.getCreateEventId()));
} finally {
versionLock_.writeLock().unlock();
}
@@ -2616,12 +2430,14 @@ public class CatalogServiceCatalog extends Catalog {
// 2) false - Table does not exist in metastore.
// 3) unknown (null) - There was exception thrown by the metastore client.
Boolean tableExistsInMetaStore;
org.apache.hadoop.hive.metastore.api.Table msTbl = null;
Db db = null;
try (MetaStoreClient msClient = getMetaStoreClient()) {
org.apache.hadoop.hive.metastore.api.Database msDb = null;
try {
tableExistsInMetaStore = msClient.getHiveClient().tableExists(dbName, tblName);
} catch (UnknownDBException e) {
msTbl = msClient.getHiveClient().getTable(dbName, tblName);
tableExistsInMetaStore = (msTbl != null);
} catch (UnknownDBException | NoSuchObjectException e) {
// The parent database does not exist in the metastore. Treat this the same
// as if the table does not exist.
tableExistsInMetaStore = false;
@@ -2667,6 +2483,7 @@ public class CatalogServiceCatalog extends Catalog {
// Add a new uninitialized table to the table cache, effectively invalidating
// any existing entry. The metadata for the table will be loaded lazily, on the
// on the next access to the table.
Preconditions.checkNotNull(msTbl);
Table newTable = addIncompleteTable(dbName, tblName);
Preconditions.checkNotNull(newTable);
if (loadInBackground_) {
@@ -3083,44 +2900,60 @@ public class CatalogServiceCatalog extends Catalog {
String partitionName = hdfsPartition == null
? HdfsTable.constructPartitionName(partitionSpec)
: hdfsPartition.getPartitionName();
LOG.info(String.format("Refreshing partition metadata: %s %s (%s)",
hdfsTable.getFullName(), partitionName, reason));
try (MetaStoreClient msClient = getMetaStoreClient()) {
org.apache.hadoop.hive.metastore.api.Partition hmsPartition = null;
try {
hmsPartition = msClient.getHiveClient().getPartition(
hdfsTable.getDb().getName(), hdfsTable.getName(), partitionName);
} catch (NoSuchObjectException e) {
// If partition does not exist in Hive Metastore, remove it from the
// catalog
if (hdfsPartition != null) {
hdfsTable.dropPartition(partitionSpec);
hdfsTable.setCatalogVersion(newCatalogVersion);
// non-existing partition was dropped from catalog, so we mark it as refreshed
wasPartitionReloaded.setRef(true);
} else {
LOG.info(String.format("Partition metadata for %s was not refreshed since "
+ "it does not exist in metastore anymore",
hdfsTable.getFullName() + " " + partitionName));
}
return hdfsTable.toTCatalogObject(resultType);
} catch (Exception e) {
throw new CatalogException("Error loading metadata for partition: "
+ hdfsTable.getFullName() + " " + partitionName, e);
}
hdfsTable.reloadPartition(msClient.getHiveClient(), hdfsPartition, hmsPartition);
}
hdfsTable.setCatalogVersion(newCatalogVersion);
wasPartitionReloaded.setRef(true);
LOG.info(String.format("Refreshed partition metadata: %s %s",
hdfsTable.getFullName(), partitionName));
return hdfsTable.toTCatalogObject(resultType);
return reloadHdfsPartition(hdfsTable, partitionName, wasPartitionReloaded,
resultType, reason, newCatalogVersion, hdfsPartition);
} finally {
Preconditions.checkState(!versionLock_.isWriteLockedByCurrentThread());
tbl.releaseWriteLock();
}
}
/**
* Reloads the HdfsPartition identified by the partitionName and returns Table's
* TCatalogObject. The returned TCatalogObject is populated based on the value of
* resultType. This method expects that the table lock has been taken already by the
* caller. If the partition does not exist in HMS and hdfsPartition is not null, this
* method will remove the HdfsPartition from the table.
*/
public TCatalogObject reloadHdfsPartition(HdfsTable hdfsTable, String partitionName,
Reference<Boolean> wasPartitionReloaded, CatalogObject.ThriftObjectType resultType,
String reason, long newCatalogVersion, @Nullable HdfsPartition hdfsPartition)
throws CatalogException {
Preconditions.checkState(hdfsTable.isWriteLockedByCurrentThread());
LOG.info(String.format("Refreshing partition metadata: %s %s (%s)",
hdfsTable.getFullName(), partitionName, reason));
try (MetaStoreClient msClient = getMetaStoreClient()) {
org.apache.hadoop.hive.metastore.api.Partition hmsPartition = null;
try {
hmsPartition = msClient.getHiveClient().getPartition(
hdfsTable.getDb().getName(), hdfsTable.getName(), partitionName);
} catch (NoSuchObjectException e) {
// If partition does not exist in Hive Metastore, remove it from the
// catalog
if (hdfsPartition != null) {
hdfsTable.dropPartition(hdfsPartition);
hdfsTable.setCatalogVersion(newCatalogVersion);
// non-existing partition was dropped from catalog, so we mark it as refreshed
wasPartitionReloaded.setRef(true);
} else {
LOG.info(String.format("Partition metadata for %s was not refreshed since "
+ "it does not exist in metastore anymore",
hdfsTable.getFullName() + " " + partitionName));
}
return hdfsTable.toTCatalogObject(resultType);
} catch (Exception e) {
throw new CatalogException("Error loading metadata for partition: "
+ hdfsTable.getFullName() + " " + partitionName, e);
}
hdfsTable.reloadPartition(msClient.getHiveClient(), hdfsPartition, hmsPartition);
}
hdfsTable.setCatalogVersion(newCatalogVersion);
wasPartitionReloaded.setRef(true);
LOG.info(String.format("Refreshed partition metadata: %s %s",
hdfsTable.getFullName(), partitionName));
return hdfsTable.toTCatalogObject(resultType);
}
public CatalogDeltaLog getDeleteLog() { return deleteLog_; }
/**
@@ -3666,4 +3499,9 @@ public class CatalogServiceCatalog extends Catalog {
ExternalEventsProcessor metastoreEventProcessor) {
this.metastoreEventProcessor_ = metastoreEventProcessor;
}
@VisibleForTesting
public void setCatalogMetastoreServer(ICatalogMetastoreServer catalogMetastoreServer) {
this.catalogMetastoreServer_ = catalogMetastoreServer;
}
}

View File

@@ -107,12 +107,24 @@ public class Db extends CatalogObjectImpl implements FeDb {
// its associated HMS operation (eg. alterDbOwner or commentOnDb)
private final ReentrantLock dbLock_ = new ReentrantLock();
// if this Db is created by catalogd and if events processing is ACTIVE then
// this field represents the event id pertaining to the creation of this database
// in hive metastore. Defaults to -1 for already existing databases or if events
// processing is disabled.
private long createEventId_ = -1;
public Db(String name, org.apache.hadoop.hive.metastore.api.Database msDb) {
setMetastoreDb(name, msDb);
tableCache_ = new CatalogObjectCache<>();
functions_ = new HashMap<>();
}
public long getCreateEventId() { return createEventId_; }
public void setCreateEventId(long eventId) {
createEventId_ = eventId;
}
public void setIsSystemDb(boolean b) { isSystemDb_ = b; }
/**

View File

@@ -683,6 +683,11 @@ public class HdfsPartition extends CatalogObjectImpl
// it's not used in coordinators.
private final InFlightEvents inFlightEvents_;
// event id in hive metastore which pertains to the creation of this partition object.
// if this partition is not created by this catalogd or if the events processing is
// not active this is set to -1.
private final long createEventId_;
/**
* Constructor. Needed for third party extensions that want to use their own builder
* to construct the object.
@@ -701,10 +706,10 @@ public class HdfsPartition extends CatalogObjectImpl
partitionKeyValues, fileFormatDescriptor, encodedFileDescriptors,
encodedInsertFileDescriptors, encodedDeleteFileDescriptors, location,
isMarkedCached, accessLevel, hmsParameters, cachedMsPartitionDescriptor,
partitionStats, hasIncrementalStats, numRows, writeId, inFlightEvents);
partitionStats, hasIncrementalStats, numRows, writeId,
inFlightEvents, /*createEventId=*/-1L);
}
protected HdfsPartition(HdfsTable table, long id, long prevId, String partName,
List<LiteralExpr> partitionKeyValues, HdfsStorageDescriptor fileFormatDescriptor,
@Nonnull ImmutableList<byte[]> encodedFileDescriptors,
@@ -714,7 +719,7 @@ public class HdfsPartition extends CatalogObjectImpl
boolean isMarkedCached, TAccessLevel accessLevel, Map<String, String> hmsParameters,
CachedHmsPartitionDescriptor cachedMsPartitionDescriptor,
byte[] partitionStats, boolean hasIncrementalStats, long numRows, long writeId,
InFlightEvents inFlightEvents) {
InFlightEvents inFlightEvents, long createEventId) {
table_ = table;
id_ = id;
prevId_ = prevId;
@@ -733,6 +738,7 @@ public class HdfsPartition extends CatalogObjectImpl
numRows_ = numRows;
writeId_ = writeId;
inFlightEvents_ = inFlightEvents;
createEventId_ = createEventId;
if (partName == null && id_ != CatalogObjectsConstants.PROTOTYPE_PARTITION_ID) {
partName_ = FeCatalogUtils.getPartitionName(this);
} else {
@@ -740,6 +746,8 @@ public class HdfsPartition extends CatalogObjectImpl
}
}
public long getCreateEventId() { return createEventId_; }
@Override // FeFsPartition
public HdfsStorageDescriptor getInputFormatDescriptor() {
return fileFormatDescriptor_;
@@ -870,7 +878,12 @@ public class HdfsPartition extends CatalogObjectImpl
Preconditions.checkState(table_.isWriteLockedByCurrentThread(),
"removeFromVersionsForInflightEvents called without holding the table lock on "
+ "partition " + getPartitionName() + " of table " + table_.getFullName());
return inFlightEvents_.remove(isInsertEvent, versionNumber);
boolean ret = inFlightEvents_.remove(isInsertEvent, versionNumber);
if (!ret) {
LOG.debug("Remove of in-flight version number failed for {}: {}", versionNumber,
inFlightEvents_.print());
}
return ret;
}
/**
@@ -884,12 +897,15 @@ public class HdfsPartition extends CatalogObjectImpl
Preconditions.checkState(table_.isWriteLockedByCurrentThread(),
"addToVersionsForInflightEvents called without holding the table lock on "
+ "partition " + getPartitionName() + " of table " + table_.getFullName());
if (!inFlightEvents_.add(isInsertEvent, versionNumber)) {
boolean added = inFlightEvents_.add(isInsertEvent, versionNumber);
if (!added) {
LOG.warn(String.format("Could not add %s version to the partition %s of table %s. "
+ "This could cause unnecessary refresh of the partition when the event is"
+ "received by the Events processor.", versionNumber, getPartitionName(),
getTable().getFullName()));
}
LOG.trace("{} {} to in-flight list {}",
(added ? "Added" : "Could not add"), versionNumber, inFlightEvents_.print());
}
@Override // FeFsPartition
@@ -1163,7 +1179,11 @@ public class HdfsPartition extends CatalogObjectImpl
private boolean hasIncrementalStats_ = false;
private long numRows_ = -1;
private long writeId_ = -1L;
private InFlightEvents inFlightEvents_ = new InFlightEvents(20);
// event id in metastore which pertains to the creation of this partition. Defaults
// to -1 if the partition was not created by this catalogd or if events processing
// is not active.
private long createEventId_ = -1L;
private InFlightEvents inFlightEvents_ = new InFlightEvents();
@Nullable
private HdfsPartition oldInstance_ = null;
@@ -1223,7 +1243,7 @@ public class HdfsPartition extends CatalogObjectImpl
fileFormatDescriptor_, encodedFileDescriptors_, encodedInsertFileDescriptors_,
encodedDeleteFileDescriptors_, location_, isMarkedCached_, accessLevel_,
hmsParameters_, cachedMsPartitionDescriptor_, partitionStats_,
hasIncrementalStats_, numRows_, writeId_, inFlightEvents_);
hasIncrementalStats_, numRows_, writeId_, inFlightEvents_, createEventId_);
}
public Builder setId(long id) {
@@ -1231,6 +1251,11 @@ public class HdfsPartition extends CatalogObjectImpl
return this;
}
public Builder setCreateEventId(long eventId) {
createEventId_ = eventId;
return this;
}
public Builder setPrevId(long prevId) {
prevId_ = prevId;
return this;
@@ -1516,8 +1541,12 @@ public class HdfsPartition extends CatalogObjectImpl
MetastoreEventPropertyKey.CATALOG_VERSION.getKey())) {
return;
}
inFlightEvents_.add(false, Long.parseLong(
boolean added = inFlightEvents_.add(false, Long.parseLong(
hmsParameters_.get(MetastoreEventPropertyKey.CATALOG_VERSION.getKey())));
LOG.trace("{} {} to inflight events {}",
(added ? "Added" : "Could not add"), Long.parseLong(
hmsParameters_.get(MetastoreEventPropertyKey.CATALOG_VERSION.getKey())),
inFlightEvents_.print());
}
private List<FileDescriptor> fdsFromThrift(List<THdfsFileDesc> tFileDescs) {

View File

@@ -42,6 +42,7 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.RemoteIterator;
import org.apache.hadoop.hive.common.FileUtils;
import org.apache.hadoop.hive.common.ValidTxnList;
import org.apache.hadoop.hive.common.ValidWriteIdList;
import org.apache.hadoop.hive.conf.HiveConf;
@@ -549,6 +550,18 @@ public class HdfsTable extends Table implements FeFsTable {
return Utils.getPartitionFromThriftPartitionSpec(table, partitionKeyValues);
}
/**
* Get the partition by the given list of partValues. Returns null if none of the
* partitions match the given list of partValues.
*/
public HdfsPartition getPartition(List<LiteralExpr> partValues) {
Preconditions.checkNotNull(partValues);
for (HdfsPartition partition: partitionMap_.values()) {
if (partValues.equals(partition.getPartitionValues())) return partition;
}
return null;
}
/**
* Gets the HdfsPartition matching the Thrift version of the partition spec.
* Returns null if no match was found.
@@ -835,20 +848,31 @@ public class HdfsTable extends Table implements FeFsTable {
* Partitions may be empty, or may not even exist in the filesystem (a partition's
* location may have been changed to a new path that is about to be created by an
* INSERT). Also loads the file metadata for this partition. Returns new partition
* if successful or null if none was created.
* if successful or null if none was created. If the map of Partition name to eventID
* is not null, it uses it to set the {@code createEventId_} of the
* HdfsPartition.
*
* Throws CatalogException if one of the supplied storage descriptors contains metadata
* that Impala can't understand.
*/
public List<HdfsPartition> createAndLoadPartitions(IMetaStoreClient client,
List<org.apache.hadoop.hive.metastore.api.Partition> msPartitions)
throws CatalogException {
List<org.apache.hadoop.hive.metastore.api.Partition> msPartitions,
@Nullable Map<String, Long> msPartitionsToEventId) throws CatalogException {
List<HdfsPartition.Builder> addedPartBuilders = new ArrayList<>();
FsPermissionCache permCache = preloadPermissionsCache(msPartitions);
for (org.apache.hadoop.hive.metastore.api.Partition partition: msPartitions) {
HdfsPartition.Builder partBuilder = createPartitionBuilder(partition.getSd(),
partition, permCache);
Preconditions.checkNotNull(partBuilder);
long eventId = -1L;
if (msPartitionsToEventId != null) {
String partName = FeCatalogUtils.getPartitionName(this, partition.getValues());
if (!msPartitionsToEventId.containsKey(partName)) {
LOG.warn("Create event id for partition {} not found. Using -1.", partName);
}
eventId = msPartitionsToEventId.getOrDefault(partName, -1L);
}
partBuilder.setCreateEventId(eventId);
addedPartBuilders.add(partBuilder);
}
loadFileMetadataForPartitions(client, addedPartBuilders, /*isRefresh=*/false);
@@ -1068,7 +1092,7 @@ public class HdfsTable extends Table implements FeFsTable {
return partition;
}
private HdfsPartition dropPartition(HdfsPartition partition) {
public HdfsPartition dropPartition(HdfsPartition partition) {
return dropPartition(partition, true);
}
@@ -1109,7 +1133,7 @@ public class HdfsTable extends Table implements FeFsTable {
throws TableLoadingException {
load(reuseMetadata, client, msTbl, /* loadPartitionFileMetadata */
true, /* loadTableSchema*/true, false,
/* partitionsToUpdate*/null, null, reason);
/* partitionsToUpdate*/null, null, null, reason);
}
public void load(boolean reuseMetadata, IMetaStoreClient client,
@@ -1117,7 +1141,7 @@ public class HdfsTable extends Table implements FeFsTable {
String reason) throws TableLoadingException {
load(reuseMetadata, client, msTbl, /* loadPartitionFileMetadata */
true, /* loadTableSchema*/true, refreshUpdatedPartitions,
/* partitionsToUpdate*/null, null, reason);
/* partitionsToUpdate*/null, null, null, reason);
}
public void load(boolean reuseMetadata, IMetaStoreClient hiveClient,
@@ -1126,7 +1150,7 @@ public class HdfsTable extends Table implements FeFsTable {
throws CatalogException {
load(reuseMetadata, hiveClient, msTbl, /* loadPartitionFileMetadata */
true, /* loadTableSchema*/true, refreshUpdatedPartitions,
/* partitionsToUpdate*/null, debugAction, reason);
/* partitionsToUpdate*/null, debugAction, null, reason);
}
/**
@@ -1157,7 +1181,8 @@ public class HdfsTable extends Table implements FeFsTable {
org.apache.hadoop.hive.metastore.api.Table msTbl,
boolean loadParitionFileMetadata, boolean loadTableSchema,
boolean refreshUpdatedPartitions, Set<String> partitionsToUpdate,
@Nullable String debugAction, String reason) throws TableLoadingException {
@Nullable String debugAction, @Nullable Map<String, Long> partitionToEventId,
String reason) throws TableLoadingException {
final Timer.Context context =
getMetrics().getTimer(Table.LOAD_DURATION_METRIC).time();
String annotation = String.format("%s metadata for %s%s partition(s) of %s.%s (%s)",
@@ -1200,7 +1225,7 @@ public class HdfsTable extends Table implements FeFsTable {
} else {
storageMetadataLoadTime_ += updatePartitionsFromHms(
client, partitionsToUpdate, loadParitionFileMetadata,
refreshUpdatedPartitions, debugAction);
refreshUpdatedPartitions, partitionToEventId, debugAction);
}
LOG.info("Incrementally loaded table metadata for: " + getFullName());
} else {
@@ -1339,17 +1364,22 @@ public class HdfsTable extends Table implements FeFsTable {
*/
private long updatePartitionsFromHms(IMetaStoreClient client,
Set<String> partitionsToUpdate, boolean loadPartitionFileMetadata,
boolean refreshUpdatedPartitions, String debugAction) throws Exception {
boolean refreshUpdatedPartitions, Map<String, Long> partitionToEventId,
String debugAction) throws Exception {
if (LOG.isTraceEnabled()) LOG.trace("Sync table partitions: " + getFullName());
org.apache.hadoop.hive.metastore.api.Table msTbl = getMetaStoreTable();
Preconditions.checkNotNull(msTbl);
Preconditions.checkState(msTbl.getPartitionKeysSize() != 0);
Preconditions.checkState(loadPartitionFileMetadata || partitionsToUpdate == null);
PartitionDeltaUpdater deltaUpdater =
refreshUpdatedPartitions ? new PartBasedDeltaUpdater(client,
loadPartitionFileMetadata, partitionsToUpdate, debugAction)
: new PartNameBasedDeltaUpdater(client, loadPartitionFileMetadata,
partitionsToUpdate, debugAction);
PartitionDeltaUpdater deltaUpdater;
if (refreshUpdatedPartitions) {
deltaUpdater = new PartBasedDeltaUpdater(client,
loadPartitionFileMetadata, partitionsToUpdate, partitionToEventId,
debugAction);
} else {
deltaUpdater = new PartNameBasedDeltaUpdater(client, loadPartitionFileMetadata,
partitionsToUpdate, partitionToEventId, debugAction);
}
deltaUpdater.apply();
return deltaUpdater.loadTimeForFileMdNs_;
}
@@ -1373,13 +1403,16 @@ public class HdfsTable extends Table implements FeFsTable {
// reloaded.
private final Set<String> partitionsToUpdate_;
private final String debugAction_;
protected final Map<String, Long> partitionToEventId_;
PartitionDeltaUpdater(IMetaStoreClient client, boolean loadPartitionFileMetadata,
Set<String> partitionsToUpdate, String debugAction) {
Set<String> partitionsToUpdate, @Nullable Map<String, Long> partitionToEventId,
String debugAction) {
this.client_ = client;
this.loadFileMd_ = loadPartitionFileMetadata;
this.partitionsToUpdate_ = partitionsToUpdate;
this.debugAction_ = debugAction;
partitionToEventId_ = partitionToEventId;
}
/**
@@ -1498,8 +1531,10 @@ public class HdfsTable extends Table implements FeFsTable {
public PartBasedDeltaUpdater(
IMetaStoreClient client, boolean loadPartitionFileMetadata,
Set<String> partitionsToUpdate, String debugAction) throws Exception {
super(client, loadPartitionFileMetadata, partitionsToUpdate, debugAction);
Set<String> partitionsToUpdate, Map<String, Long> partitionToEventId,
String debugAction) throws Exception {
super(client, loadPartitionFileMetadata, partitionsToUpdate, partitionToEventId,
debugAction);
Stopwatch sw = Stopwatch.createStarted();
List<Partition> partitionList;
if (partitionsToUpdate != null) {
@@ -1570,7 +1605,7 @@ public class HdfsTable extends Table implements FeFsTable {
}
}
return loadPartitionsFromMetastore(newMsPartitions,
/*inprogressPartBuilders=*/null, client_);
/*inprogressPartBuilders=*/null, partitionToEventId_, client_);
}
@Override
@@ -1581,7 +1616,10 @@ public class HdfsTable extends Table implements FeFsTable {
updatedPartitions.add(Preconditions
.checkNotNull(msPartitions_.get(partName)));
}
return loadPartitionsFromMetastore(updatedPartitions, updatedPartBuilders, client_);
// we pass partitionToEventId argument as null below because updated partitions
// partitions were preexisting before load and just modified from outside.
return loadPartitionsFromMetastore(updatedPartitions, updatedPartBuilders,
null, client_);
}
}
@@ -1597,8 +1635,10 @@ public class HdfsTable extends Table implements FeFsTable {
public PartNameBasedDeltaUpdater(
IMetaStoreClient client, boolean loadPartitionFileMetadata,
Set<String> partitionsToUpdate, String debugAction) throws Exception {
super(client, loadPartitionFileMetadata, partitionsToUpdate, debugAction);
Set<String> partitionsToUpdate, Map<String, Long> partitionToEventId,
String debugAction) throws Exception {
super(client, loadPartitionFileMetadata, partitionsToUpdate, partitionToEventId,
debugAction);
// Retrieve all the partition names from the Hive Metastore. We need this to
// identify the delta between partitions of the local HdfsTable and the table entry
// in the Hive Metastore. Note: This is a relatively "cheap" operation
@@ -1625,14 +1665,16 @@ public class HdfsTable extends Table implements FeFsTable {
addedPartNames.addAll(Sets
.difference(partitionNamesFromHms_, knownPartitionNames));
return loadPartitionsFromMetastore(addedPartNames,
/*inprogressPartBuilders=*/null, client_);
/*inprogressPartBuilders=*/null, partitionToEventId_, client_);
}
@Override
public long loadUpdatedPartitions(
Map<String, HdfsPartition.Builder> updatedPartitionBuilders) throws Exception {
// we pass partitionToEventId argument as null below because updated partitions
// partitions were preexisting before load and just modified from outside.
return loadPartitionsFromMetastore(updatedPartitionBuilders.keySet(),
updatedPartitionBuilders, client_);
updatedPartitionBuilders, null, client_);
}
}
@@ -1763,7 +1805,8 @@ public class HdfsTable extends Table implements FeFsTable {
* @return time in nanoseconds spent in loading file metadata.
*/
private long loadPartitionsFromMetastore(Set<String> partitionNames,
Map<String, HdfsPartition.Builder> inprogressPartBuilders, IMetaStoreClient client)
Map<String, HdfsPartition.Builder> inprogressPartBuilders,
@Nullable Map<String, Long> partitionToEventId, IMetaStoreClient client)
throws Exception {
Preconditions.checkNotNull(partitionNames);
if (partitionNames.isEmpty()) return 0;
@@ -1771,11 +1814,13 @@ public class HdfsTable extends Table implements FeFsTable {
List<Partition> msPartitions = new ArrayList<>(
MetaStoreUtil.fetchPartitionsByName(
client, Lists.newArrayList(partitionNames), db_.getName(), name_));
return loadPartitionsFromMetastore(msPartitions, inprogressPartBuilders, client);
return loadPartitionsFromMetastore(msPartitions, inprogressPartBuilders,
partitionToEventId, client);
}
private long loadPartitionsFromMetastore(List<Partition> msPartitions,
Map<String, HdfsPartition.Builder> inprogressPartBuilders, IMetaStoreClient client)
Map<String, HdfsPartition.Builder> inprogressPartBuilders,
@Nullable Map<String, Long> partitionToEventId, IMetaStoreClient client)
throws Exception {
FsPermissionCache permCache = preloadPermissionsCache(msPartitions);
List<HdfsPartition.Builder> partBuilders = new ArrayList<>(msPartitions.size());
@@ -1789,6 +1834,9 @@ public class HdfsTable extends Table implements FeFsTable {
}
partBuilder = createOrUpdatePartitionBuilder(
msPartition.getSd(), msPartition, permCache, partBuilder);
if (partitionToEventId != null) {
partBuilder.setCreateEventId(partitionToEventId.getOrDefault(partName, -1L));
}
partBuilders.add(partBuilder);
}
long fileMdLoadTime = loadFileMetadataForPartitions(client, partBuilders,
@@ -2398,6 +2446,29 @@ public class HdfsTable extends Table implements FeFsTable {
}
}
/**
* Returns a list of {@link LiteralExpr} which is type compatible with the partition
* keys of this table. This is useful to convert the string values which are received
* from metastore events to {@link LiteralExpr}.
*/
public List<LiteralExpr> getTypeCompatiblePartValues(List<String> values)
throws UnsupportedEncodingException {
List<LiteralExpr> result = new ArrayList<>();
List<Column> partitionColumns = getClusteringColumns();
Preconditions.checkState(partitionColumns.size() == values.size());
for (int i=0; i<partitionColumns.size(); ++i) {
Pair<String, LiteralExpr> pair = getPartitionExprFromValue(values.get(i),
partitionColumns.get(i).getType());
if (pair == null) {
LOG.error("Could not get a type compatible value for key {} with value {}", i,
values.get(i));
return null;
}
result.add(pair.second);
}
return result;
}
/**
* Checks that the last component of 'path' is of the form "<partitionkey>=<v>"
* where 'v' is a type-compatible value from the domain of the 'partitionKey' column.
@@ -2414,9 +2485,23 @@ public class HdfsTable extends Table implements FeFsTable {
Column column = getColumn(partName[0]);
Preconditions.checkNotNull(column);
Type type = column.getType();
LiteralExpr expr = null;
return getPartitionExprFromValue(partName[1], type);
}
/**
* Converts a given partition value to a {@link LiteralExpr} based on the type of the
* partition column.
* @param partValue Value of the partition column
* @param type Type of the partition column
* @return Pair which contains the partition value and its equivalent
* {@link LiteralExpr} according to the type provided.
* @throws UnsupportedEncodingException
*/
private Pair<String, LiteralExpr> getPartitionExprFromValue(String partValue, Type type)
throws UnsupportedEncodingException {
LiteralExpr expr;
// URL decode the partition value since it may contain encoded URL.
String value = URLDecoder.decode(partName[1], StandardCharsets.UTF_8.name());
String value = URLDecoder.decode(partValue, StandardCharsets.UTF_8.name());
if (!value.equals(getNullPartitionKeyValue())) {
try {
expr = LiteralExpr.createFromUnescapedStr(value, type);
@@ -2438,7 +2523,7 @@ public class HdfsTable extends Table implements FeFsTable {
} else {
expr = new NullLiteral();
}
return new Pair<String, LiteralExpr>(value, expr);
return new Pair<>(value, expr);
}
@Override // FeFsTable

View File

@@ -263,7 +263,8 @@ public class IcebergTable extends Table implements FeIcebergTable {
// Loading hdfs table after loaded schema from Iceberg,
// in case we create external Iceberg table skipping column info in sql.
icebergFileFormat_ = Utils.getIcebergFileFormat(msTbl);
hdfsTable_.load(false, msClient, msTable_, true, true, false, null, null, reason);
hdfsTable_
.load(false, msClient, msTable_, true, true, false, null, null,null, reason);
pathHashToFileDescMap_ = Utils.loadAllPartition(this);
loadAllColumnStats(msClient);
} catch (Exception e) {

View File

@@ -137,6 +137,10 @@ public abstract class Table extends CatalogObjectImpl implements FeTable {
// impalad.
protected long lastUsedTime_;
// Represents the event id in the metastore which pertains to the creation of this
// table. Defaults to -1 for a preexisting table or if events processing is not active.
protected long createEventId_ = -1;
// tracks the in-flight metastore events for this table. Used by Events processor to
// avoid unnecessary refresh when the event is received
private final InFlightEvents inFlightEvents = new InFlightEvents();
@@ -163,6 +167,9 @@ public abstract class Table extends CatalogObjectImpl implements FeTable {
public static final String LOAD_DURATION_ALL_COLUMN_STATS =
"load-duration.all-column-stats";
// metric key for the number of in-flight events for this table.
public static final String NUMBER_OF_INFLIGHT_EVENTS = "num-inflight-events";
// Table property key for storing the time of the last DDL operation.
public static final String TBL_PROP_LAST_DDL_TIME = "transient_lastDdlTime";
@@ -187,6 +194,11 @@ public abstract class Table extends CatalogObjectImpl implements FeTable {
initMetrics();
}
public long getCreateEventId() { return createEventId_; }
public void setCreateEventId(long eventId) {
this.createEventId_ = eventId;
}
/**
* Returns if the given HMS table is an external table (uses table type if
* available or else uses table properties). Implementation is based on org.apache
@@ -324,6 +336,7 @@ public abstract class Table extends CatalogObjectImpl implements FeTable {
metrics_.addTimer(LOAD_DURATION_STORAGE_METADATA);
metrics_.addTimer(HMS_LOAD_TBL_SCHEMA);
metrics_.addTimer(LOAD_DURATION_ALL_COLUMN_STATS);
metrics_.addCounter(NUMBER_OF_INFLIGHT_EVENTS);
}
public Metrics getMetrics() { return metrics_; }
@@ -871,7 +884,11 @@ public abstract class Table extends CatalogObjectImpl implements FeTable {
Preconditions.checkState(isWriteLockedByCurrentThread(),
"removeFromVersionsForInFlightEvents called without taking the table lock on "
+ getFullName());
return inFlightEvents.remove(isInsertEvent, versionNumber);
boolean removed = inFlightEvents.remove(isInsertEvent, versionNumber);
if (removed) {
metrics_.getCounter(NUMBER_OF_INFLIGHT_EVENTS).dec();
}
return removed;
}
/**
@@ -895,6 +912,8 @@ public abstract class Table extends CatalogObjectImpl implements FeTable {
LOG.warn(String.format("Could not add %s version to the table %s. This could "
+ "cause unnecessary refresh of the table when the event is received by the "
+ "Events processor.", versionNumber, getFullName()));
} else {
metrics_.getCounter(NUMBER_OF_INFLIGHT_EVENTS).inc();
}
}

View File

@@ -17,10 +17,16 @@
package org.apache.impala.catalog;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.hive.metastore.IMetaStoreClient.NotificationFilter;
import org.apache.hadoop.hive.metastore.TableType;
import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
import org.apache.hadoop.hive.metastore.api.NotificationEvent;
import org.apache.impala.catalog.events.MetastoreEvents.CreateTableEvent;
import org.apache.impala.catalog.events.MetastoreEventsProcessor;
import org.apache.impala.compat.MetastoreShim;
import org.apache.log4j.Logger;
@@ -28,6 +34,7 @@ import com.google.common.base.Stopwatch;
import org.apache.impala.catalog.MetaStoreClientPool.MetaStoreClient;
import org.apache.impala.util.ThreadNameAnnotator;
import org.apache.thrift.TException;
/**
* Class that implements the logic for how a table's metadata should be loaded from
@@ -49,17 +56,20 @@ public class TableLoader {
/**
* Creates the Impala representation of Hive/HBase metadata for one table.
* Calls load() on the appropriate instance of Table subclass.
* Calls load() on the appropriate instance of Table subclass. If the eventId is not
* negative, it fetches all events from metastore to find out a CREATE_TABLE
* event from metastore which is used to set the createEventId of the table.
* Returns new instance of Table, If there were any errors loading the table metadata
* an IncompleteTable will be returned that contains details on the error.
*/
public Table load(Db db, String tblName, String reason) {
public Table load(Db db, String tblName, long eventId, String reason) {
Stopwatch sw = Stopwatch.createStarted();
String fullTblName = db.getName() + "." + tblName;
String annotation = "Loading metadata for: " + fullTblName + " (" + reason + ")";
LOG.info(annotation);
Table table;
// turn all exceptions into TableLoadingException
List<NotificationEvent> events = null;
try (ThreadNameAnnotator tna = new ThreadNameAnnotator(annotation);
MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
org.apache.hadoop.hive.metastore.api.Table msTbl = null;
@@ -68,6 +78,26 @@ public class TableLoader {
synchronized (metastoreAccessLock_) {
msTbl = msClient.getHiveClient().getTable(db.getName(), tblName);
}
if (eventId != -1 && catalog_.isEventProcessingActive()) {
// If the eventId is not -1 it means this table was likely created by Impala.
// However, since the load operation of the table can happen much later, it is
// possible that the table was recreated outside Impala and hence the eventId
// which is stored in the loaded table needs to be updated to the latest.
// we are only interested in fetching the events if we have a valid eventId
// for a table. For tables where eventId is unknown are not created by
// this catalogd and hence the self-event detection logic does not apply.
events = MetastoreEventsProcessor.getNextMetastoreEvents(catalog_, eventId,
notificationEvent -> CreateTableEvent.CREATE_TABLE_EVENT_TYPE
.equals(notificationEvent.getEventType())
&& notificationEvent.getDbName().equalsIgnoreCase(db.getName())
&& notificationEvent.getTableName().equalsIgnoreCase(tblName));
}
if (events != null && !events.isEmpty()) {
// if the table was recreated after the table was initially created in the
// catalogd, we should move the eventId forward to the latest create_table
// event.
eventId = events.get(events.size() - 1).getEventId();
}
long hmsLoadTime = hmsLoadSW.elapsed(TimeUnit.NANOSECONDS);
// Check that the Hive TableType is supported
TableType tableType = TableType.valueOf(msTbl.getTableType());
@@ -83,6 +113,7 @@ public class TableLoader {
"Unrecognized table type for table: " + fullTblName);
}
table.updateHMSLoadTableSchemaTime(hmsLoadTime);
table.setCreateEventId(eventId);
table.load(false, msClient.getHiveClient(), msTbl, reason);
table.validate();
} catch (TableLoadingException e) {

View File

@@ -230,7 +230,8 @@ public class TableLoadingMgr {
* the same underlying loading task (Future) will be used, helping to prevent duplicate
* loads of the same table.
*/
public LoadRequest loadAsync(final TTableName tblName, final String reason)
public LoadRequest loadAsync(final TTableName tblName, final long createdEventId,
final String reason)
throws DatabaseNotFoundException {
final Db parentDb = catalog_.getDb(tblName.getDb_name());
if (parentDb == null) {
@@ -241,7 +242,7 @@ public class TableLoadingMgr {
FutureTask<Table> tableLoadTask = new FutureTask<Table>(new Callable<Table>() {
@Override
public Table call() throws Exception {
return tblLoader_.load(parentDb, tblName.table_name, reason);
return tblLoader_.load(parentDb, tblName.table_name, createdEventId, reason);
}});
FutureTask<Table> existingValue = loadingTables_.putIfAbsent(tblName, tableLoadTask);

View File

@@ -0,0 +1,148 @@
// 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.impala.catalog.events;
import com.google.common.base.Preconditions;
import java.util.List;
import java.util.SortedMap;
import java.util.TreeMap;
import org.apache.hadoop.hive.common.FileUtils;
import org.apache.hadoop.hive.metastore.api.Database;
import org.apache.hadoop.hive.metastore.api.Table;
import org.apache.impala.catalog.HdfsTable;
import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEvent;
import org.apache.impala.service.CatalogOpExecutor;
/**
* This class is used to store delete events for catalog objects (database, tables and
* partitions). These delete events are added when an object is removed from the catalogd
* so that when subsequent {@link MetastoreEvent} is received by the
* {@link MetastoreEventsProcessor} it can ignore it. The {@link CatalogOpExecutor}
* adds the delete events to the log and the {@link MetastoreEventsProcessor} garbage
* collects the delete events when the event is received from the metastore.
*/
public class DeleteEventLog {
private SortedMap<Long, Object> eventLog_;
//TODO add catalogName in this key when we support metastore catalogs
// key format for databases "DB:DbName"
private static final String DB_KEY_FORMAT_STR = "DB:%s";
//TODO add catalogName in this key when we support metastore catalogs
// key format for tables "TBL:DbName.tblName"
private static final String TBL_KEY_FORMAT_STR = "TBL:%s.%s";
// TODO Add catalog name here.
// key format for partitions "PART:FullTblName.partName"
private static final String PART_KEY_FORMAT_STR = "PART:%s.%s";
public DeleteEventLog() {
eventLog_ = new TreeMap<>();
}
/**
* Generic util method to add an object in the DeleteEventLog.
* @param eventId The eventId from hive metastore which maps to this deletion event.
* @param value The value of the DeleteEventLog entry to be stored.
*/
public synchronized void addRemovedObject(long eventId, Object value) {
Preconditions.checkNotNull(value);
eventLog_.put(eventId, value);
}
/**
* Util method to determine if the given object was removed since the given eventId.
* @param eventId The eventId after which which we are interested to find the deletion
* event.
* @param value The value of the delete entry.
* @return True if there exists a delete entry with an event which is strictly greater
* that eventId and value which is equal to the given value. False otherwise.
*/
public synchronized boolean wasRemovedAfter(long eventId, Object value) {
Preconditions.checkNotNull(value);
return keyExistsAfterEventId(eventId, value);
}
private boolean keyExistsAfterEventId(long eventId, Object key) {
for (Object objectName : eventLog_.tailMap(eventId + 1).values()) {
if (key.equals(objectName)) {
return true;
}
}
return false;
}
/**
* Removes all the delete log entries which are less than or equal to the given
* eventId.
*/
public synchronized void garbageCollect(long eventId) {
if (!eventLog_.isEmpty() && eventLog_.firstKey() <= eventId) {
eventLog_ = new TreeMap<>(eventLog_.tailMap(eventId + 1));
}
}
/**
* Returns the current size of the delete event log.
*/
public synchronized int size() {
return eventLog_.size();
}
/**
* Util method to generate a key for database deletion entry value.
*/
public static String getDbKey(String dbName) {
return String
.format(DB_KEY_FORMAT_STR, dbName).toLowerCase();
}
/**
* Util method to generate a key for table deletion entry value.
*/
public static String getTblKey(String dbName, String tblName) {
return String.format(TBL_KEY_FORMAT_STR, dbName, tblName).toLowerCase();
}
/**
* Returns the partition key to be used to add a Partition delete log entry.
* @param hdfsTable The HdfsTable which the partition belongs to.
* @param partValues The partition values which represent the partition.
* @return String value to be used in DeleteEventLog for the given partition
* identifiers.
*/
public static String getPartitionKey(HdfsTable hdfsTable, List<String> partValues) {
return String.format(PART_KEY_FORMAT_STR, hdfsTable.getFullName(),
FileUtils.makePartName(hdfsTable.getClusteringColNames(), partValues));
}
/**
* Util method to generate a database deletion entry value from a
* {@link Database} object.
*/
public static String getKey(Database database) {
return getDbKey(database.getName());
}
/**
* Util method to generate a table deletion entry value from a {@link Table} object.
*/
public static String getKey(Table tbl) {
return String
.format(TBL_KEY_FORMAT_STR, tbl.getDbName(), tbl.getTableName())
.toLowerCase();
}
}

View File

@@ -0,0 +1,36 @@
// 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.impala.catalog.events;
import org.apache.hadoop.hive.metastore.api.NotificationEvent;
import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEvent;
/**
* Factory interface to generate a {@link MetastoreEvent} from a {@link NotificationEvent}
* object.
*/
public interface EventFactory {
/**
* Generates a {@link MetastoreEvent} representing {@link NotificationEvent}
* @param hmsEvent the event as received from Hive Metastore.
* @return {@link MetastoreEvent} representing hmsEvent.
* @throws MetastoreNotificationException If the hmsEvent information cannot be parsed.
*/
MetastoreEvent get(NotificationEvent hmsEvent) throws MetastoreNotificationException;
}

View File

@@ -18,8 +18,11 @@
package org.apache.impala.catalog.events;
import org.apache.impala.catalog.CatalogException;
import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEventFactory;
import org.apache.impala.service.CatalogOpExecutor;
import org.apache.impala.thrift.TEventProcessorMetrics;
import org.apache.impala.thrift.TEventProcessorMetricsSummaryResponse;
import org.apache.kudu.client.Delete;
/**
* Interface to process external events
@@ -71,4 +74,15 @@ public interface ExternalEventsProcessor {
* content of a dedicated page for the event processor
*/
TEventProcessorMetricsSummaryResponse getEventProcessorSummary();
/**
* Gets the {@link MetastoreEventFactory} to be used for creating
* {@link MetastoreEvents.MetastoreEvent}.
*/
EventFactory getEventsFactory() throws MetastoreNotificationException;
/**
* Gets the delete event log for this events processor.
*/
DeleteEventLog getDeleteEventLog();
}

View File

@@ -17,6 +17,7 @@
package org.apache.impala.catalog.events;
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import java.util.LinkedList;
@@ -32,7 +33,7 @@ import org.slf4j.LoggerFactory;
public class InFlightEvents {
// maximum number of catalog versions to store for in-flight events for this table
private static final int DEFAULT_MAX_NUMBER_OF_INFLIGHT_EVENTS = 10;
private static final int DEFAULT_MAX_NUMBER_OF_INFLIGHT_EVENTS = 100;
// maximum number of eventIds to store for in-flight events for this table
private static final int DEFAULT_MAX_NUMBER_OF_INFLIGHT_INSERT_EVENTS = 100;
@@ -59,11 +60,6 @@ public class InFlightEvents {
this.capacity_for_eventIds_ = DEFAULT_MAX_NUMBER_OF_INFLIGHT_INSERT_EVENTS;
}
public InFlightEvents(int capacity) {
Preconditions.checkState(capacity > 0);
this.capacity_for_versions_ = capacity;
this.capacity_for_eventIds_ = DEFAULT_MAX_NUMBER_OF_INFLIGHT_INSERT_EVENTS;
}
/**
* Gets the current list of versions for in-flight events for this table
* @param isInsertEvent if true, return list of eventIds for in-flight Insert events
@@ -142,4 +138,12 @@ public class InFlightEvents {
return versionsForInflightEvents_.size();
}
}
/**
* String representation of the current InFlightEvents. Useful for logging and debugging
* purposes.
*/
public String print() {
return Joiner.on(',').join(versionsForInflightEvents_);
}
}

View File

@@ -18,6 +18,7 @@
package org.apache.impala.catalog.events;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Joiner;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
@@ -33,7 +34,7 @@ import java.util.Map;
import org.apache.hadoop.hive.metastore.api.Database;
import org.apache.hadoop.hive.metastore.api.NotificationEvent;
import org.apache.hadoop.hive.metastore.api.Partition;
import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
import org.apache.hadoop.hive.metastore.api.Table;
import org.apache.hadoop.hive.metastore.messaging.AddPartitionMessage;
import org.apache.hadoop.hive.metastore.messaging.AlterPartitionMessage;
import org.apache.hadoop.hive.metastore.messaging.CreateTableMessage;
@@ -48,15 +49,13 @@ import org.apache.impala.analysis.TableName;
import org.apache.impala.catalog.CatalogException;
import org.apache.impala.catalog.CatalogServiceCatalog;
import org.apache.impala.catalog.DatabaseNotFoundException;
import org.apache.impala.catalog.Db;
import org.apache.impala.catalog.HdfsTable;
import org.apache.impala.catalog.Table;
import org.apache.impala.catalog.TableLoadingException;
import org.apache.impala.catalog.TableNotFoundException;
import org.apache.impala.catalog.TableNotLoadedException;
import org.apache.impala.common.Metrics;
import org.apache.impala.common.Pair;
import org.apache.impala.common.Reference;
import org.apache.impala.service.CatalogOpExecutor;
import org.apache.impala.thrift.TPartitionKeyValue;
import org.apache.impala.thrift.TTableName;
import org.apache.impala.util.AcidUtils;
@@ -137,7 +136,7 @@ public class MetastoreEvents {
/**
* Factory class to create various MetastoreEvents.
*/
public static class MetastoreEventFactory {
public static class MetastoreEventFactory implements EventFactory {
private static final Logger LOG =
LoggerFactory.getLogger(MetastoreEventFactory.class);
@@ -146,9 +145,12 @@ public class MetastoreEvents {
private final CatalogServiceCatalog catalog_;
// metrics registry to be made available for each events to publish metrics
private final Metrics metrics_;
// catalogOpExecutor needed for the create/drop events for table and database.
private final CatalogOpExecutor catalogOpExecutor_;
public MetastoreEventFactory(CatalogServiceCatalog catalog, Metrics metrics) {
this.catalog_ = Preconditions.checkNotNull(catalog);
public MetastoreEventFactory(CatalogOpExecutor catalogOpExecutor, Metrics metrics) {
this.catalogOpExecutor_ = Preconditions.checkNotNull(catalogOpExecutor);
this.catalog_ = Preconditions.checkNotNull(catalogOpExecutor.getCatalog());
this.metrics_ = Preconditions.checkNotNull(metrics);
}
@@ -156,25 +158,35 @@ public class MetastoreEvents {
* creates instance of <code>MetastoreEvent</code> used to process a given event type.
* If the event type is unknown, returns a IgnoredEvent
*/
private MetastoreEvent get(NotificationEvent event)
public MetastoreEvent get(NotificationEvent event)
throws MetastoreNotificationException {
Preconditions.checkNotNull(event.getEventType());
MetastoreEventType metastoreEventType =
MetastoreEventType.from(event.getEventType());
switch (metastoreEventType) {
case CREATE_TABLE: return new CreateTableEvent(catalog_, metrics_, event);
case DROP_TABLE: return new DropTableEvent(catalog_, metrics_, event);
case ALTER_TABLE: return new AlterTableEvent(catalog_, metrics_, event);
case CREATE_DATABASE: return new CreateDatabaseEvent(catalog_, metrics_, event);
case DROP_DATABASE: return new DropDatabaseEvent(catalog_, metrics_, event);
case ALTER_DATABASE: return new AlterDatabaseEvent(catalog_, metrics_, event);
case ADD_PARTITION: return new AddPartitionEvent(catalog_, metrics_, event);
case DROP_PARTITION: return new DropPartitionEvent(catalog_, metrics_, event);
case ALTER_PARTITION: return new AlterPartitionEvent(catalog_, metrics_, event);
case INSERT: return new InsertEvent(catalog_, metrics_, event);
case CREATE_TABLE:
return new CreateTableEvent(catalogOpExecutor_, metrics_, event);
case DROP_TABLE:
return new DropTableEvent(catalogOpExecutor_, metrics_, event);
case ALTER_TABLE:
return new AlterTableEvent(catalogOpExecutor_, metrics_, event);
case CREATE_DATABASE:
return new CreateDatabaseEvent(catalogOpExecutor_, metrics_, event);
case DROP_DATABASE:
return new DropDatabaseEvent(catalogOpExecutor_, metrics_, event);
case ALTER_DATABASE:
return new AlterDatabaseEvent(catalogOpExecutor_, metrics_, event);
case ADD_PARTITION:
return new AddPartitionEvent(catalogOpExecutor_, metrics_, event);
case DROP_PARTITION:
return new DropPartitionEvent(catalogOpExecutor_, metrics_, event);
case ALTER_PARTITION:
return new AlterPartitionEvent(catalogOpExecutor_, metrics_, event);
case INSERT:
return new InsertEvent(catalogOpExecutor_, metrics_, event);
default:
// ignore all the unknown events by creating a IgnoredEvent
return new IgnoredEvent(catalog_, metrics_, event);
return new IgnoredEvent(catalogOpExecutor_, metrics_, event);
}
}
@@ -221,12 +233,6 @@ public class MetastoreEvents {
+ "blacklisted database or table %s", blacklistedObject));
metastoreEvents.remove(i);
numFilteredEvents++;
} else if (currentEvent.isRemovedAfter(metastoreEvents.subList(i + 1,
metastoreEvents.size()))) {
LOG.info(currentEvent.debugString("Filtering out this event since the object is "
+ "either removed or renamed later in the event stream"));
metastoreEvents.remove(i);
numFilteredEvents++;
} else {
i++;
}
@@ -235,6 +241,8 @@ public class MetastoreEvents {
+ "filtered out: %d", sizeBefore, numFilteredEvents));
metrics_.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC)
.inc(numFilteredEvents);
LOG.info("Incremented skipped metric to " + metrics_
.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).getCount());
return metastoreEvents;
}
}
@@ -256,6 +264,8 @@ public class MetastoreEvents {
// CatalogServiceCatalog instance on which the event needs to be acted upon
protected final CatalogServiceCatalog catalog_;
protected final CatalogOpExecutor catalogOpExecutor_;
// the notification received from metastore which is processed by this
protected final NotificationEvent event_;
@@ -281,9 +291,10 @@ public class MetastoreEvents {
protected final Metrics metrics_;
MetastoreEvent(CatalogServiceCatalog catalogServiceCatalog, Metrics metrics,
MetastoreEvent(CatalogOpExecutor catalogOpExecutor, Metrics metrics,
NotificationEvent event) {
this.catalog_ = catalogServiceCatalog;
this.catalogOpExecutor_ = catalogOpExecutor;
this.catalog_ = catalogOpExecutor_.getCatalog();
this.event_ = event;
this.eventId_ = event_.getEventId();
this.eventType_ = MetastoreEventType.from(event.getEventType());
@@ -294,6 +305,8 @@ public class MetastoreEvents {
this.metrics_ = metrics;
}
public long getEventId() { return eventId_; }
public String getDbName() { return dbName_; }
public String getTableName() { return tblName_; }
@@ -309,6 +322,8 @@ public class MetastoreEvents {
if (isEventProcessingDisabled()) {
LOG.info(debugString("Skipping this event because of flag evaluation"));
metrics_.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).inc();
infoLog("Incremented skipped metric to " + metrics_
.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).getCount());
return;
}
process();
@@ -428,11 +443,15 @@ public class MetastoreEvents {
protected boolean isSelfEvent(boolean isInsertEvent) {
try {
if (catalog_.evaluateSelfEvent(isInsertEvent, getSelfEventContext())) {
metrics_.getCounter(MetastoreEventsProcessor.NUMBER_OF_SELF_EVENTS).inc();
metrics_.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).inc();
infoLog("Incremented events skipped counter to {}",
metrics_.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC)
.getCount());
return true;
}
} catch (CatalogException e) {
debugLog("Received exception {}. Ignoring self-event evaluation", e.getMessage());
debugLog("Received exception {}. Ignoring self-event evaluation",
e.getMessage());
}
return false;
}
@@ -457,9 +476,9 @@ public class MetastoreEvents {
// case of alter events
protected org.apache.hadoop.hive.metastore.api.Table msTbl_;
private MetastoreTableEvent(CatalogServiceCatalog catalogServiceCatalog,
private MetastoreTableEvent(CatalogOpExecutor catalogOpExecutor,
Metrics metrics, NotificationEvent event) {
super(catalogServiceCatalog, metrics, event);
super(catalogOpExecutor, metrics, event);
Preconditions.checkNotNull(dbName_, debugString("Database name cannot be null"));
tblName_ = Preconditions.checkNotNull(event.getTableName());
if (MetastoreEventType.OTHER.equals(eventType_)) {
@@ -559,11 +578,21 @@ public class MetastoreEvents {
*/
protected boolean reloadTableFromCatalog(String operation, boolean isTransactional)
throws CatalogException {
if (!catalog_.reloadTableIfExists(dbName_, tblName_,
"Processing " + operation + " event from HMS")) {
debugLog("Automatic refresh on table {} failed as the table "
+ "either does not exist anymore or is not in loaded state.",
getFullyQualifiedTblName());
try {
if (!catalog_.reloadTableIfExists(dbName_, tblName_,
"Processing " + operation + " event from HMS")) {
debugLog("Automatic refresh on table {} failed as the table "
+ "either does not exist anymore or is not in loaded state.",
getFullyQualifiedTblName());
return false;
}
} catch (TableLoadingException | DatabaseNotFoundException e) {
// there could be many reasons for receiving a tableLoading exception,
// eg. table doesn't exist in HMS anymore or table schema is not supported
// or Kudu threw an exception due to some internal error. There is not much
// we can do here other than log it appropriately.
debugLog("Table {} was not refreshed due to error {}",
getFullyQualifiedTblName(), e.getMessage());
return false;
}
String tblStr = isTransactional ? "transactional table" : "table";
@@ -574,35 +603,28 @@ public class MetastoreEvents {
/**
* Refreshes a partition provided by given spec only if the table is loaded
* @param tPartSpec
* @param partition the Partition object which needs to be reloaded.
* @param reason Event type which caused the refresh, used for logging by catalog
* @return false if the table or database did not exist or was not loaded, else
* returns true.
* @throws CatalogException
*/
protected boolean reloadPartition(List<TPartitionKeyValue> tPartSpec, String reason)
protected boolean reloadPartition(Partition partition, String reason)
throws CatalogException {
try {
boolean result = catalog_.reloadPartitionIfExists(dbName_,
tblName_, tPartSpec, reason);
if (!result) {
debugLog("partition {} on table {} was not refreshed since it does not exist "
+ "in catalog anymore", HdfsTable.constructPartitionName(tPartSpec),
getFullyQualifiedTblName());
} else {
boolean result = catalogOpExecutor_
.reloadPartitionIfExists(eventId_, dbName_, tblName_, partition, reason);
if (result) {
metrics_.getCounter(MetastoreEventsProcessor.NUMBER_OF_PARTITION_REFRESHES)
.inc();
infoLog("Table {} partition {} has been refreshed", getFullyQualifiedTblName(),
HdfsTable.constructPartitionName(tPartSpec));
}
return true;
return result;
} catch (TableNotLoadedException e) {
debugLog("Partition {} on table {} was not refreshed since it is not loaded",
HdfsTable.constructPartitionName(tPartSpec), getFullyQualifiedTblName());
debugLog("Ignoring the event since table {} is not loaded",
getFullyQualifiedTblName());
} catch (DatabaseNotFoundException | TableNotFoundException e) {
debugLog("Refresh of table {} partition {} "
+ "event failed as the database or table is not present in the catalog.",
getFullyQualifiedTblName(), HdfsTable.constructPartitionName(tPartSpec));
debugLog("Ignoring the event since table {} is not found",
getFullyQualifiedTblName());
}
return false;
}
@@ -612,9 +634,9 @@ public class MetastoreEvents {
* Base class for all the database events
*/
public static abstract class MetastoreDatabaseEvent extends MetastoreEvent {
MetastoreDatabaseEvent(CatalogServiceCatalog catalogServiceCatalog, Metrics metrics,
MetastoreDatabaseEvent(CatalogOpExecutor catalogOpExecutor, Metrics metrics,
NotificationEvent event) {
super(catalogServiceCatalog, metrics, event);
super(catalogOpExecutor, metrics, event);
Preconditions.checkNotNull(dbName_, debugString("Database name cannot be null"));
debugLog("Creating event {} of type {} on database {}", eventId_,
eventType_, dbName_);
@@ -639,12 +661,14 @@ public class MetastoreEvents {
* MetastoreEvent for CREATE_TABLE event type
*/
public static class CreateTableEvent extends MetastoreTableEvent {
public static final String CREATE_TABLE_EVENT_TYPE = "CREATE_TABLE";
/**
* Prevent instantiation from outside should use MetastoreEventFactory instead
*/
private CreateTableEvent(CatalogServiceCatalog catalog, Metrics metrics,
private CreateTableEvent(CatalogOpExecutor catalogOpExecutor, Metrics metrics,
NotificationEvent event) throws MetastoreNotificationException {
super(catalog, metrics, event);
super(catalogOpExecutor, metrics, event);
Preconditions.checkArgument(MetastoreEventType.CREATE_TABLE.equals(eventType_));
Preconditions
.checkNotNull(event.getMessage(), debugString("Event message is null"));
@@ -679,22 +703,21 @@ public class MetastoreEvents {
// a self-event (see description of self-event in the class documentation of
// MetastoreEventsProcessor)
try {
if (!catalog_.addTableIfNotExists(dbName_, tblName_)) {
debugLog(
"Not adding the table {} since it already exists in catalog", tblName_);
return;
if (catalogOpExecutor_.addTableIfNotRemovedLater(eventId_, msTbl_)) {
infoLog("Successfully added table {}", getFullyQualifiedTblName());
metrics_.getCounter(MetastoreEventsProcessor.NUMBER_OF_TABLES_ADDED).inc();
} else {
metrics_.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).inc();
infoLog("Incremented skipped metric to " + metrics_
.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).getCount());
}
} catch (CatalogException e) {
// if a exception is thrown, it could be due to the fact that the db did not
// exist in the catalog cache. This could only happen if the previous
// create_database event for this table errored out
throw new MetastoreNotificationException(debugString(
"Unable to add table while processing for table %s because the "
+ "database doesn't exist. This could be due to a previous error while "
+ "processing CREATE_DATABASE event for the database %s",
getFullyQualifiedTblName(), dbName_), e);
// if a DatabaseNotFoundException is caught here it means either we incorrectly
// determined that the event needs to be processed instead of skipped, or we
// somehow missed the previous create database event.
throw new MetastoreNotificationException(
debugString("Unable to process event", e));
}
debugLog("Added a table {}", getFullyQualifiedTblName());
}
@Override
@@ -730,6 +753,10 @@ public class MetastoreEvents {
}
return false;
}
public Table getTable() {
return msTbl_;
}
}
/**
@@ -745,9 +772,9 @@ public class MetastoreEvents {
* Prevent instantiation from outside should use MetastoreEventFactory instead
*/
@VisibleForTesting
InsertEvent(CatalogServiceCatalog catalog, Metrics metrics,
InsertEvent(CatalogOpExecutor catalogOpExecutor, Metrics metrics,
NotificationEvent event) throws MetastoreNotificationException {
super(catalog, metrics, event);
super(catalogOpExecutor, metrics, event);
Preconditions.checkArgument(MetastoreEventType.INSERT.equals(eventType_));
InsertMessage insertMessage =
MetastoreEventsProcessor.getMessageDeserializer()
@@ -800,19 +827,17 @@ public class MetastoreEvents {
private void processPartitionInserts() throws MetastoreNotificationException {
// For partitioned table, refresh the partition only.
Preconditions.checkNotNull(insertPartition_);
List<TPartitionKeyValue> tPartSpec = getTPartitionSpecFromHmsPartition(msTbl_,
insertPartition_);
try {
// Ignore event if table or database is not in catalog. Throw exception if
// refresh fails. If the partition does not exist in metastore the reload
// method below removes it from the catalog
reloadPartition(tPartSpec, "INSERT");
reloadPartition(insertPartition_, "INSERT");
} catch (CatalogException e) {
throw new MetastoreNotificationNeedsInvalidateException(debugString("Refresh "
+ "partition on table {} partition {} failed. Event processing cannot "
+ "continue. Issue an invalidate metadata command to reset the event "
+ "processor state.", getFullyQualifiedTblName(),
HdfsTable.constructPartitionName(tPartSpec)), e);
Joiner.on(',').join(insertPartition_.getValues())), e);
}
}
@@ -823,21 +848,12 @@ public class MetastoreEvents {
// For non-partitioned tables, refresh the whole table.
Preconditions.checkArgument(insertPartition_ == null);
try {
// Ignore event if table or database is not in the catalog. Throw exception if
// refresh fails.
reloadTableFromCatalog("INSERT", false);
} catch (CatalogException e) {
if (e instanceof TableLoadingException &&
e.getCause() instanceof NoSuchObjectException) {
LOG.warn(
"Ignoring the refresh of the table since the table does"
+ " not exist in metastore anymore");
} else {
throw new MetastoreNotificationNeedsInvalidateException(
debugString("Refresh table {} failed. Event processing "
+ "cannot continue. Issue an invalidate metadata command to reset "
+ "the event processor state.", getFullyQualifiedTblName()), e);
}
throw new MetastoreNotificationNeedsInvalidateException(
debugString("Refresh table {} failed. Event processing "
+ "cannot continue. Issue an invalidate metadata command to reset "
+ "the event processor state.", getFullyQualifiedTblName()), e);
}
}
}
@@ -862,9 +878,9 @@ public class MetastoreEvents {
* Prevent instantiation from outside should use MetastoreEventFactory instead
*/
@VisibleForTesting
AlterTableEvent(CatalogServiceCatalog catalog, Metrics metrics,
AlterTableEvent(CatalogOpExecutor catalogOpExecutor, Metrics metrics,
NotificationEvent event) throws MetastoreNotificationException {
super(catalog, metrics, event);
super(catalogOpExecutor, metrics, event);
Preconditions.checkArgument(MetastoreEventType.ALTER_TABLE.equals(eventType_));
JSONAlterTableMessage alterTableMessage =
(JSONAlterTableMessage) MetastoreEventsProcessor.getMessageDeserializer()
@@ -888,12 +904,38 @@ public class MetastoreEvents {
MetastoreEventPropertyKey.DISABLE_EVENT_HMS_SYNC.getKey()));
}
public boolean isRename() { return isRename_; }
public Table getBeforeTable() { return tableBefore_; }
public Table getAfterTable() { return tableAfter_; }
@Override
protected SelfEventContext getSelfEventContext() {
return new SelfEventContext(tableAfter_.getDbName(), tableAfter_.getTableName(),
tableAfter_.getParameters());
}
private void processRename() throws CatalogException {
if (!isRename_) return;
Reference<Boolean> oldTblRemoved = new Reference<>();
Reference<Boolean> newTblAdded = new Reference<>();
catalogOpExecutor_
.renameTableFromEvent(eventId_, tableBefore_, tableAfter_, oldTblRemoved,
newTblAdded);
if (oldTblRemoved.getRef()) {
metrics_.getCounter(MetastoreEventsProcessor.NUMBER_OF_TABLES_REMOVED).inc();
}
if (newTblAdded.getRef()) {
metrics_.getCounter(MetastoreEventsProcessor.NUMBER_OF_TABLES_ADDED).inc();
}
if (!oldTblRemoved.getRef() || !newTblAdded.getRef()) {
metrics_.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).inc();
infoLog("Incremented skipped metric to " + metrics_
.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).getCount());
}
}
/**
* If the ALTER_TABLE event is due a table rename, this method removes the old table
* and creates a new table with the new name. Else, this just issues a refresh
@@ -901,6 +943,10 @@ public class MetastoreEvents {
*/
@Override
public void process() throws MetastoreNotificationException, CatalogException {
if (isRename_) {
processRename();
return;
}
// Determine whether this is an event which we have already seen or if it is a new
// event
if (isSelfEvent()) {
@@ -917,48 +963,22 @@ public class MetastoreEvents {
// in case of table level alters from external systems it is better to do a full
// refresh eg. this could be due to as simple as adding a new parameter or a
// full blown adding or changing column type
// detect the special case where a table is renamed
if (!isRename_) {
// table is not renamed, need to refresh the table if its loaded
if (!reloadTableFromCatalog("ALTER_TABLE", false)) {
if (wasEventSyncTurnedOn()) {
// we received this alter table event on a non-existing table. We also
// detect that event sync was turned on in this event. This may mean that
// the table creation was skipped earlier because event sync was turned off
// we don't really know how many of events we have skipped till now because
// the sync was disabled all this while before we receive such a event. We
// error on the side of caution by stopping the event processing and
// letting the user to issue a invalidate metadata to reset the state
throw new MetastoreNotificationNeedsInvalidateException(debugString(
"Detected that event sync was turned on for the table %s "
+ "and the table does not exist. Event processing cannot be "
+ "continued further. Issue a invalidate metadata command to reset "
+ "the event processing state", getFullyQualifiedTblName()));
}
// rename is already handled above
if (!reloadTableFromCatalog("ALTER_TABLE", false)) {
if (wasEventSyncTurnedOn()) {
// we received this alter table event on a non-existing table. We also
// detect that event sync was turned on in this event. This may mean that
// the table creation was skipped earlier because event sync was turned off
// we don't really know how many of events we have skipped till now because
// the sync was disabled all this while before we receive such a event. We
// error on the side of caution by stopping the event processing and
// letting the user to issue a invalidate metadata to reset the state
throw new MetastoreNotificationNeedsInvalidateException(debugString(
"Detected that event sync was turned on for the table %s "
+ "and the table does not exist. Event processing cannot be "
+ "continued further. Issue a invalidate metadata command to reset "
+ "the event processing state", getFullyQualifiedTblName()));
}
return;
}
// table was renamed, remove the old table
infoLog("Found that {} table was renamed. Renaming it by "
+ "remove and adding a new table",
new TableName(msTbl_.getDbName(), msTbl_.getTableName()));
TTableName oldTTableName =
new TTableName(msTbl_.getDbName(), msTbl_.getTableName());
TTableName newTTableName =
new TTableName(tableAfter_.getDbName(), tableAfter_.getTableName());
// Table is renamed if old db and table exist in catalog. If the rename is to a
// different database, we check if this other database exists in catalog. If
// either the old table, old database or new database are not in catalog, we skip
// this event.
if (!catalog_.renameTableIfExists(oldTTableName, newTTableName)) {
debugLog("Did not remove old table to rename table {} to {} since "
+ "it does not exist anymore or either the old database or the new "
+ "database don't exist anymore.", qualify(oldTTableName),
qualify(newTTableName));
} else {
infoLog("Renamed old table {} to new table {}.", qualify(oldTTableName),
qualify(newTTableName));
}
}
@@ -1026,12 +1046,14 @@ public class MetastoreEvents {
*/
public static class DropTableEvent extends MetastoreTableEvent {
public static final String DROP_TABLE_EVENT_TYPE = "DROP_TABLE";
/**
* Prevent instantiation from outside should use MetastoreEventFactory instead
*/
private DropTableEvent(CatalogServiceCatalog catalog, Metrics metrics,
private DropTableEvent(CatalogOpExecutor catalogOpExecutor, Metrics metrics,
NotificationEvent event) throws MetastoreNotificationException {
super(catalog, metrics, event);
super(catalogOpExecutor, metrics, event);
Preconditions.checkArgument(MetastoreEventType.DROP_TABLE.equals(eventType_));
JSONDropTableMessage dropTableMessage =
(JSONDropTableMessage) MetastoreEventsProcessor.getMessageDeserializer()
@@ -1065,23 +1087,21 @@ public class MetastoreEvents {
* not a huge problem since the tables will eventually be created when the
* create events are processed but there will be a non-zero amount of time when the
* table will not be existing in catalog.
* TODO : Once HIVE-21595 is available we should rely on table_id for determining a
* newer incarnation of a previous table.
*/
@Override
public void process() {
Reference<Boolean> tblWasFound = new Reference<>();
Reference<Boolean> tblMatched = new Reference<>();
Table removedTable = catalog_.removeTableIfExists(msTbl_, tblWasFound, tblMatched);
if (removedTable != null) {
infoLog("Removed table {} ", getFullyQualifiedTblName());
} else if (!tblWasFound.getRef()) {
debugLog("Table {} was not removed since it does not exist in catalog anymore.",
tblName_);
} else if (!tblMatched.getRef()) {
infoLog(debugString("Table %s was not removed from "
+ "catalog since the creation time of the table did not match", tblName_));
public void process() throws MetastoreNotificationException {
Reference<Boolean> tblRemovedLater = new Reference<>();
boolean removedTable;
removedTable = catalogOpExecutor_
.removeTableIfNotAddedLater(eventId_, msTbl_.getDbName(),
msTbl_.getTableName(), tblRemovedLater);
if (removedTable) {
infoLog("Successfully removed table {}", getFullyQualifiedTblName());
metrics_.getCounter(MetastoreEventsProcessor.NUMBER_OF_TABLES_REMOVED).inc();
} else {
metrics_.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).inc();
infoLog("Incremented skipped metric to " + metrics_
.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).getCount());
}
}
}
@@ -1091,15 +1111,16 @@ public class MetastoreEvents {
*/
public static class CreateDatabaseEvent extends MetastoreDatabaseEvent {
public static final String CREATE_DATABASE_EVENT_TYPE = "CREATE_DATABASE";
// metastore database object as parsed from NotificationEvent message
private final Database createdDatabase_;
/**
* Prevent instantiation from outside should use MetastoreEventFactory instead
*/
private CreateDatabaseEvent(CatalogServiceCatalog catalog, Metrics metrics,
private CreateDatabaseEvent(CatalogOpExecutor catalogOpExecutor, Metrics metrics,
NotificationEvent event) throws MetastoreNotificationException {
super(catalog, metrics, event);
super(catalogOpExecutor, metrics, event);
Preconditions.checkArgument(MetastoreEventType.CREATE_DATABASE.equals(eventType_));
JSONCreateDatabaseMessage createDatabaseMessage =
(JSONCreateDatabaseMessage) MetastoreEventsProcessor.getMessageDeserializer()
@@ -1116,6 +1137,8 @@ public class MetastoreEvents {
}
}
public Database getDatabase() { return createdDatabase_; }
@Override
public SelfEventContext getSelfEventContext() {
throw new UnsupportedOperationException("Self-event evaluation is unnecessary for"
@@ -1128,33 +1151,20 @@ public class MetastoreEvents {
*/
@Override
public void process() {
// if the database already exists in catalog, by definition, it is a later version
// of the database since metastore will not allow it be created if it was already
// existing at the time of creation. In such case, it is safe to assume that the
// already existing database in catalog is a later version with the same name and
// this event can be ignored
if (catalog_.addDbIfNotExists(dbName_, createdDatabase_)) {
infoLog("Successfully added database {}", dbName_);
boolean dbAdded = catalogOpExecutor_
.addDbIfNotRemovedLater(eventId_, createdDatabase_);
if (!dbAdded) {
debugLog(
"Database {} was not added since it either exists or was "
+ "removed since the event was generated", dbName_);
metrics_.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).inc();
infoLog("Incremented skipped metric to " + metrics_
.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).getCount());
} else {
infoLog("Database {} already exists", dbName_);
debugLog("Successfully added database {}", dbName_);
metrics_.getCounter(MetastoreEventsProcessor.NUMBER_OF_DATABASES_ADDED).inc();
}
}
@Override
public boolean isRemovedAfter(List<MetastoreEvent> events) {
Preconditions.checkNotNull(events);
for (MetastoreEvent event : events) {
if (event.eventType_.equals(MetastoreEventType.DROP_DATABASE)) {
DropDatabaseEvent dropDatabaseEvent = (DropDatabaseEvent) event;
if (dbName_.equalsIgnoreCase(dropDatabaseEvent.dbName_)) {
infoLog("Found database {} is removed later in event {} of type {} ", dbName_,
dropDatabaseEvent.eventId_, dropDatabaseEvent.eventType_);
return true;
}
}
}
return false;
}
}
/**
@@ -1167,9 +1177,9 @@ public class MetastoreEvents {
/**
* Prevent instantiation from outside should use MetastoreEventFactory instead
*/
private AlterDatabaseEvent(CatalogServiceCatalog catalog, Metrics metrics,
private AlterDatabaseEvent(CatalogOpExecutor catalogOpExecutor, Metrics metrics,
NotificationEvent event) throws MetastoreNotificationException {
super(catalog, metrics, event);
super(catalogOpExecutor, metrics, event);
Preconditions.checkArgument(MetastoreEventType.ALTER_DATABASE.equals(eventType_));
JSONAlterDatabaseMessage alterDatabaseMessage =
(JSONAlterDatabaseMessage) MetastoreEventsProcessor.getMessageDeserializer()
@@ -1188,7 +1198,7 @@ public class MetastoreEvents {
* the Db object from the event
*/
@Override
public void process() throws CatalogException, MetastoreNotificationException {
public void process() throws CatalogException {
if (isSelfEvent()) {
infoLog("Not processing the event as it is a self-event");
return;
@@ -1216,16 +1226,16 @@ public class MetastoreEvents {
*/
public static class DropDatabaseEvent extends MetastoreDatabaseEvent {
public static final String DROP_DATABASE_EVENT_TYPE = "DROP_DATABASE";
// Metastore database object as parsed from NotificationEvent message
private final Database droppedDatabase_;
/**
* Prevent instantiation from outside should use MetastoreEventFactory instead
*/
private DropDatabaseEvent(
CatalogServiceCatalog catalog, Metrics metrics, NotificationEvent event)
CatalogOpExecutor catalogOpExecutor, Metrics metrics, NotificationEvent event)
throws MetastoreNotificationException {
super(catalog, metrics, event);
super(catalogOpExecutor, metrics, event);
Preconditions.checkArgument(MetastoreEventType.DROP_DATABASE.equals(eventType_));
JSONDropDatabaseMessage dropDatabaseMessage =
(JSONDropDatabaseMessage) MetastoreEventsProcessor.getMessageDeserializer()
@@ -1264,23 +1274,18 @@ public class MetastoreEvents {
* since the databases will eventually be created when the create events are
* processed but there will be a non-zero amount of time when the database will not
* be existing in catalog.
* TODO : Once HIVE-21595 is available we should rely on database_id for determining a
* newer incarnation of a previous database.
*/
@Override
public void process() {
Reference<Boolean> dbFound = new Reference<>();
Reference<Boolean> dbMatched = new Reference<>();
Db removedDb = catalog_.removeDbIfExists(droppedDatabase_, dbFound, dbMatched);
if (removedDb != null) {
boolean dbRemoved = catalogOpExecutor_
.removeDbIfNotAddedLater(eventId_, droppedDatabase_);
if (dbRemoved) {
infoLog("Removed Database {} ", dbName_);
} else if (!dbFound.getRef()) {
debugLog("Database {} was not removed since it " +
"did not exist in catalog.", dbName_);
} else if (!dbMatched.getRef()) {
infoLog(debugString("Database %s was not removed from catalog since "
+ "the creation time of the Database did not match", dbName_));
metrics_.getCounter(MetastoreEventsProcessor.NUMBER_OF_DATABASES_REMOVED).inc();
} else {
metrics_.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).inc();
infoLog("Incremented skipped metric to " + metrics_
.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).getCount());
}
}
}
@@ -1314,15 +1319,17 @@ public class MetastoreEvents {
}
}
public static class AddPartitionEvent extends MetastoreTableEvent {
public static final String ADD_PARTITION_EVENT_TYPE = "ADD_PARTITION";
private final List<Partition> addedPartitions_;
private final List<List<TPartitionKeyValue>> partitionKeyVals_;
/**
* Prevent instantiation from outside should use MetastoreEventFactory instead
*/
private AddPartitionEvent(CatalogServiceCatalog catalog, Metrics metrics,
private AddPartitionEvent(CatalogOpExecutor catalogOpExecutor, Metrics metrics,
NotificationEvent event) throws MetastoreNotificationException {
super(catalog, metrics, event);
super(catalogOpExecutor, metrics, event);
Preconditions.checkState(eventType_.equals(MetastoreEventType.ADD_PARTITION));
if (event.getMessage() == null) {
throw new IllegalStateException(debugString("Event message is null"));
@@ -1347,15 +1354,21 @@ public class MetastoreEvents {
@Override
public SelfEventContext getSelfEventContext() {
Map<String, String> params = new HashMap<>();
// all the partitions are added as one transaction and hence we expect all the
// added partitions to have the same catalog service identifiers. Using the first
// one for the params is enough for the purpose of self-event evaluation
if (!addedPartitions_.isEmpty()) {
params.putAll(addedPartitions_.get(0).getParameters());
// self event evaluation is only done for transactional tables currently.
// for non-transactional tables we use the partition level createEventId
if (AcidUtils.isTransactionalTable(msTbl_.getParameters())) {
Map<String, String> params = new HashMap<>();
// all the partitions are added as one transaction and hence we expect all the
// added partitions to have the same catalog service identifiers. Using the first
// one for the params is enough for the purpose of self-event evaluation
if (!addedPartitions_.isEmpty()) {
params.putAll(addedPartitions_.get(0).getParameters());
}
return new SelfEventContext(dbName_, tblName_, partitionKeyVals_,
params);
}
return new SelfEventContext(dbName_, tblName_, partitionKeyVals_,
params);
throw new UnsupportedOperationException("Self-event evaluation is unnecessary for"
+ " this event type");
}
@Override
@@ -1365,13 +1378,9 @@ public class MetastoreEvents {
infoLog("Partition list is empty. Ignoring this event.");
return;
}
if (isSelfEvent()) {
infoLog("Not processing the event as it is a self-event");
return;
}
try {
// Reload the whole table if it's a transactional table.
if (AcidUtils.isTransactionalTable(msTbl_.getParameters())) {
if (AcidUtils.isTransactionalTable(msTbl_.getParameters()) && !isSelfEvent()) {
reloadTableFromCatalog("ADD_PARTITION", true);
} else {
// HMS adds partitions in a transactional way. This means there may be multiple
@@ -1379,13 +1388,18 @@ public class MetastoreEvents {
// by refreshing all those partitions in a loop. If any partition refresh fails,
// we throw MetastoreNotificationNeedsInvalidateException exception. We skip
// refresh of the partitions if the table is not present in the catalog.
infoLog("Trying to refresh {} partitions added to table {} in the event",
addedPartitions_.size(), getFullyQualifiedTblName());
//TODO refresh all the partition together instead of looping one by one
for (Partition partition : addedPartitions_) {
List<TPartitionKeyValue> tPartSpec =
getTPartitionSpecFromHmsPartition(msTbl_, partition);
if (!reloadPartition(tPartSpec, "ADD_PARTITION")) break;
int numPartsAdded = catalogOpExecutor_
.addPartitionsIfNotRemovedLater(eventId_, dbName_, tblName_,
addedPartitions_, "ADD_PARTITION");
if (numPartsAdded != 0) {
infoLog("Successfully added {} partitions to table {}",
numPartsAdded, getFullyQualifiedTblName());
metrics_.getCounter(MetastoreEventsProcessor.NUMBER_OF_PARTITIONS_ADDED)
.inc(numPartsAdded);
} else {
metrics_.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).inc();
infoLog("Incremented skipped metric to " + metrics_
.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).getCount());
}
}
} catch (CatalogException e) {
@@ -1395,6 +1409,10 @@ public class MetastoreEvents {
+ "processor.", getFullyQualifiedTblName()), e);
}
}
public List<Partition> getPartitions() {
return addedPartitions_;
}
}
public static class AlterPartitionEvent extends MetastoreTableEvent {
@@ -1406,9 +1424,9 @@ public class MetastoreEvents {
/**
* Prevent instantiation from outside should use MetastoreEventFactory instead
*/
private AlterPartitionEvent(CatalogServiceCatalog catalog, Metrics metrics,
private AlterPartitionEvent(CatalogOpExecutor catalogOpExecutor, Metrics metrics,
NotificationEvent event) throws MetastoreNotificationException {
super(catalog, metrics, event);
super(catalogOpExecutor, metrics, event);
Preconditions.checkState(eventType_.equals(MetastoreEventType.ALTER_PARTITION));
Preconditions.checkNotNull(event.getMessage());
AlterPartitionMessage alterPartitionMessage =
@@ -1451,7 +1469,7 @@ public class MetastoreEvents {
List<TPartitionKeyValue> tPartSpec = getTPartitionSpecFromHmsPartition(msTbl_,
partitionAfter_);
try {
reloadPartition(tPartSpec, "ALTER_PARTITION");
reloadPartition(partitionAfter_, "ALTER_PARTITION");
} catch (CatalogException e) {
throw new MetastoreNotificationNeedsInvalidateException(debugString("Refresh "
+ "partition on table {} partition {} failed. Event processing cannot "
@@ -1486,13 +1504,14 @@ public class MetastoreEvents {
public static class DropPartitionEvent extends MetastoreTableEvent {
private final List<Map<String, String>> droppedPartitions_;
public static final String EVENT_TYPE = "DROP_PARTITION";
/**
* Prevent instantiation from outside should use MetastoreEventFactory instead
*/
private DropPartitionEvent(CatalogServiceCatalog catalog, Metrics metrics,
private DropPartitionEvent(CatalogOpExecutor catalogOpExecutor, Metrics metrics,
NotificationEvent event) throws MetastoreNotificationException {
super(catalog, metrics, event);
super(catalogOpExecutor, metrics, event);
Preconditions.checkState(eventType_.equals(MetastoreEventType.DROP_PARTITION));
Preconditions.checkNotNull(event.getMessage());
DropPartitionMessage dropPartitionMessage =
@@ -1511,6 +1530,10 @@ public class MetastoreEvents {
}
}
public List<Map<String, String>> getDroppedPartitions() {
return droppedPartitions_;
}
@Override
public void process() throws MetastoreNotificationException, CatalogException {
// we have seen cases where a add_partition event is generated with empty
@@ -1520,21 +1543,24 @@ public class MetastoreEvents {
infoLog("Partition list is empty. Ignoring this event.");
}
try {
// Reload the whole table if it's a transactional table.
// Reload the whole table if it's a transactional table. In case of transactional
// tables we rely on the self-event evaluation since there is no fine-grained
// partition level refresh.
if (AcidUtils.isTransactionalTable(msTbl_.getParameters())) {
reloadTableFromCatalog("DROP_PARTITION", true);
} else {
// We refresh all the partitions that were dropped from HMS. If a refresh
// fails, we throw a MetastoreNotificationNeedsInvalidateException
infoLog("{} partitions dropped from table {}. Refreshing the partitions "
+ "to remove them from catalog.", droppedPartitions_.size(),
getFullyQualifiedTblName());
for (Map<String, String> partSpec : droppedPartitions_) {
List<TPartitionKeyValue> tPartSpec = new ArrayList<>(partSpec.size());
for (Map.Entry<String, String> entry : partSpec.entrySet()) {
tPartSpec.add(new TPartitionKeyValue(entry.getKey(), entry.getValue()));
}
if (!reloadPartition(tPartSpec, "DROP_PARTITION")) break;
int numPartsRemoved = catalogOpExecutor_
.removePartitionsIfNotAddedLater(eventId_, dbName_, tblName_,
droppedPartitions_, "DROP_PARTITION");
if (numPartsRemoved > 0) {
infoLog("{} partitions dropped from table {}", numPartsRemoved,
getFullyQualifiedTblName());
metrics_.getCounter(MetastoreEventsProcessor.NUMBER_OF_PARTITIONS_REMOVED)
.inc(numPartsRemoved);
} else {
metrics_.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).inc();
infoLog("Incremented skipped metric to " + metrics_
.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).getCount());
}
}
} catch (CatalogException e) {
@@ -1561,8 +1587,8 @@ public class MetastoreEvents {
* Prevent instantiation from outside should use MetastoreEventFactory instead
*/
private IgnoredEvent(
CatalogServiceCatalog catalog, Metrics metrics, NotificationEvent event) {
super(catalog, metrics, event);
CatalogOpExecutor catalogOpExecutor, Metrics metrics, NotificationEvent event) {
super(catalogOpExecutor, metrics, event);
}
@Override

View File

@@ -31,7 +31,9 @@ import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;
import javax.annotation.Nullable;
import org.apache.hadoop.hive.metastore.IMetaStoreClient;
import org.apache.hadoop.hive.metastore.IMetaStoreClient.NotificationFilter;
import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
import org.apache.hadoop.hive.metastore.api.NotificationEvent;
import org.apache.hadoop.hive.metastore.api.NotificationEventResponse;
@@ -42,8 +44,12 @@ import org.apache.impala.catalog.MetaStoreClientPool.MetaStoreClient;
import org.apache.impala.catalog.events.ConfigValidator.ValidationResult;
import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEvent;
import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEventFactory;
import org.apache.impala.common.ImpalaRuntimeException;
import org.apache.impala.common.Metrics;
import org.apache.impala.compat.MetastoreShim;
import org.apache.impala.service.CatalogOpExecutor;
import org.apache.impala.thrift.TCreateTableParams;
import org.apache.impala.thrift.TDdlExecResponse;
import org.apache.impala.thrift.TEventProcessorMetrics;
import org.apache.impala.thrift.TEventProcessorMetricsSummaryResponse;
import org.apache.impala.util.MetaStoreUtil;
@@ -97,38 +103,47 @@ import org.slf4j.LoggerFactory;
* corresponding object of the event could either be stale, exactly-same or at a version
* which is higher than one provided by event. Catalog state should only be updated when
* it is stale with respect to the event. In order to determine if the catalog object is
* stale, we rely on a combination of creationTime and object version. A object in catalog
* is stale if and only if its creationTime is < creationTime of the object from event OR
* its version < version from event if createTime matches
* stale, we rely on a combination of create EventId and object version.
* In case of create/drop events on database/table/partitions we use the
* <code>createEventId</code> field of the corresponding object in the catalogd
* to determine if the event needs to be processed or ignored. E.g. if Impala creates
* a table, {@link CatalogOpExecutor} will create the table and assign the createEventId
* of the table by fetching the CREATE_TABLE event from HMS. Later on when the event
* is fetched by events processor, it uses the createEventId of the Catalogd's table to
* ignore the event. Similar approach is used for databases and partition create events.
*
* If the object has the same createTime and version when compared to event or if the
* createTime > createTime from the event, the event can be safely ignored.
* In case of Drop events for database/table/partition events processor looks at the
* {@link DeleteEventLog} in the CatalogOpExecutor to determine if the table has been
* dropped already from catalogd.
*
* Following table shows the actions to be taken when the catalog state is stale.
* In case of ALTER/INSERT events events processor relies on the object version in the
* properties of the table to determine if this is a self-event or not.
*
* Following table shows the actions to be taken when the given event type is received.
*
* <pre>
* +----------------------------------------+
* | Catalog object state |
* +----------------------------+------------+------------+
* | Event type | Loaded | Incomplete | Not present|
* | | | | |
* +------------------------------------------------------+
* | | | | |
* | CREATE EVENT| removeAndAdd | Ignore | Add |
* | | | | |
* | | | | |
* | ALTER EVENT | Invalidate | Ignore | Ignore |
* | | | | |
* | | | | |
* | DROP EVENT | Remove | Remove | Ignore |
* | | | | |
* | | | | |
* | INSERT EVENT| Refresh | Ignore | Ignore |
* | | | | |
* +-------------+--------------+------------+------------+
* +------------------------------------------------+ --------------------+
* | Catalog object state |
* +--------------------------------------+-----------------------+---------------------+
* | Event type | Loaded | Incomplete | Not present |
* | | | | |
* +--------------------------------------------------------------+---------------------+
* | | | | |
* | CREATE EVENT| Ignore | Ignore | addIfNotRemovedLater|
* | | | | |
* | | | | |
* | ALTER EVENT | Refresh | Ignore | Ignore |
* | | | | |
* | | | | |
* | DROP EVENT | removeIfNotAddedLater | removeIfNotAddedLater | Ignore |
* | | | | |
* | | | | |
* | INSERT EVENT| Refresh | Ignore | Ignore |
* | | | | |
* +-------------+------------------------+-----------------------+---------------------+
* </pre>
*
* Currently event handlers rely on creation time on Database, Table and Partition to
* Currently event handlers rely on createEventId on Database, Table and Partition to
* uniquely determine if the object from event is same as object in the catalog. This
* information is used to make sure that we are deleting the right incarnation of the
* object when compared to Metastore.
@@ -141,27 +156,29 @@ import org.slf4j.LoggerFactory;
* clears this version when the corresponding version number identified by serviceId is
* received in the event. This is needed since it is possible that a external
* non-Impala system which generates the event presents the same serviceId and version
* number later on. The algorithm to detect a self-event is as below.
* number later on. The algorithm to detect such self-event is as below.
*
* 1. Add the service id and expected catalog version to table/partition parameters
* when executing the DDL operation. When the HMS operation is successful, add the
* version number to the list of version for in-flight events at table level.
* 1. Add the service id and expected catalog version to database/table/partition
* parameters when executing the DDL operation. When the HMS operation is successful, add
* the version number to the list of version for in-flight events at
* table/database/partition level.
* 2. When the event is received, the first time you see the combination of serviceId
* and version number, event processor clears the version number from table's list and
* determines the event as self-generated (and hence ignored)
* determines the event as self-generated (and hence ignored).
* 3. If the event data presents a unknown serviceId or if the version number is not
* present in the list of in-flight versions, event is not a self-event and needs to be
* processed.
*
* In order to limit the total memory footprint, only 10 version numbers are stored at
* the table. Since the event processor is expected to poll every few seconds this
* should be a reasonable bound which satisfies most use-cases. Otherwise, event
* In order to limit the total memory footprint, only 100 version numbers are stored at
* the catalog object. Since the event processor is expected to poll every few seconds
* this should be a reasonable bound which satisfies most use-cases. Otherwise, event
* processor may wrongly process a self-event to invalidate the table. In such a case,
* its a performance penalty not a correctness issue.
*
* All the operations which change the state of catalog cache while processing a certain
* event type must be atomic in nature. We rely on taking a write lock on version object
* in CatalogServiceCatalog to make sure that readers are blocked while the metadata
* event type must be atomic in nature. We rely on taking a DDL lock in CatalogOpExecutor
* in case of create/drop events and object (Db or table) level writeLock in case of alter
* events to make sure that readers are blocked while the metadata
* update operation is being performed. Since the events are generated post-metastore
* operations, such catalog updates do not need to update the state in Hive Metastore.
*
@@ -170,6 +187,12 @@ import org.slf4j.LoggerFactory;
* and no subsequent events are polled. In such a case a invalidate metadata command
* restarts the event polling which updates the lastSyncedEventId to the latest from
* metastore.
*
* TODO:
* 1. a global invalidate metadata command to get the events processor out of error state
* is too heavy weight. We should make it easier to recover from the error state.
* 2. The createEventId logic can be extended to track the last eventId which the table
* has synced to and we can then get rid of self-event logic for alter events too.
*/
public class MetastoreEventsProcessor implements ExternalEventsProcessor {
@@ -196,18 +219,56 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
public static final String EVENTS_PROCESS_DURATION_METRIC = "events-apply-duration";
// rate of events received per unit time
public static final String EVENTS_RECEIVED_METRIC = "events-received";
// total number of events which are skipped because of the flag setting
// total number of events which are skipped because of the flag setting or
// in case of [CREATE|DROP] events on [DATABASE|TABLE|PARTITION] which were ignored
// because the [DATABASE|TABLE|PARTITION] was already [PRESENT|ABSENT] in the catalogd.
public static final String EVENTS_SKIPPED_METRIC = "events-skipped";
// name of the event processor status metric
public static final String STATUS_METRIC = "status";
// last synced event id
public static final String LAST_SYNCED_ID_METRIC = "last-synced-event-id";
// metric name which counts the number of self-events which are skipped
public static final String NUMBER_OF_SELF_EVENTS = "self-events-skipped";
// metric name for number of tables which are refreshed by event processor so far
public static final String NUMBER_OF_TABLE_REFRESHES = "tables-refreshed";
// number of times events processor refreshed a partition
public static final String NUMBER_OF_PARTITION_REFRESHES = "partitions-refreshed";
// number of tables which were added to the catalogd based on events.
public static final String NUMBER_OF_TABLES_ADDED = "tables-added";
// number of tables which were removed to the catalogd based on events.
public static final String NUMBER_OF_TABLES_REMOVED = "tables-removed";
// number of databases which were added to the catalogd based on events.
public static final String NUMBER_OF_DATABASES_ADDED = "databases-added";
// number of database which were removed to the catalogd based on events.
public static final String NUMBER_OF_DATABASES_REMOVED = "databases-removed";
// number of partitions which were added to the catalogd based on events.
public static final String NUMBER_OF_PARTITIONS_ADDED = "partitions-added";
// number of partitions which were removed to the catalogd based on events.
public static final String NUMBER_OF_PARTITIONS_REMOVED = "partitions-removed";
// number of entries in the delete event log
public static final String DELETE_EVENT_LOG_SIZE = "delete-event-log-size";
/**
* Gets the next list of {@link NotificationEvent} from Hive Metastore which are
* greater than the given eventId and filtered according to the provided filter.
* @param catalog The CatalogServiceCatalog used to get the metastore client
* @param eventId The eventId after which the events are needed.
* @param filter The {@link NotificationFilter} used to filter the list of fetched
* events. Note that this is a client side filter not a server side
* filter. Unfortunately, HMS doesn't provide a similar mechanism to
* do server side filtering.
* @return List of {@link NotificationEvent} which are all greater than eventId and
* satisfy the given filter.
* @throws ImpalaRuntimeException in case of RPC errors to metastore.
*/
public static List<NotificationEvent> getNextMetastoreEvents(
CatalogServiceCatalog catalog, long eventId, NotificationFilter filter)
throws ImpalaRuntimeException {
try (MetaStoreClient msc = catalog.getMetaStoreClient()) {
return msc.getHiveClient().getNextNotification(eventId, -1, filter).getEvents();
} catch (TException e) {
throw new ImpalaRuntimeException(String.format(
CatalogOpExecutor.HMS_RPC_ERROR_FORMAT_STR, "getNextNotification"), e);
}
}
// possible status of event processor
public enum EventProcessorStatus {
@@ -245,14 +306,19 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
// have to pass it around as a argument in constructor in MetastoreEvents
private final Metrics metrics_ = new Metrics();
// When events processing is ACTIVE this delete event log is used to keep track of
// DROP events for databases, tables and partitions so that the MetastoreEventsProcessor
// can ignore the drop events when they are received later.
private final DeleteEventLog deleteEventLog_ = new DeleteEventLog();
@VisibleForTesting
MetastoreEventsProcessor(CatalogServiceCatalog catalog, long startSyncFromId,
MetastoreEventsProcessor(CatalogOpExecutor catalogOpExecutor, long startSyncFromId,
long pollingFrequencyInSec) throws CatalogException {
Preconditions.checkState(pollingFrequencyInSec > 0);
this.catalog_ = Preconditions.checkNotNull(catalog);
this.catalog_ = Preconditions.checkNotNull(catalogOpExecutor.getCatalog());
validateConfigs();
lastSyncedEventId_.set(startSyncFromId);
metastoreEventFactory_ = new MetastoreEventFactory(catalog_, metrics_);
metastoreEventFactory_ = new MetastoreEventFactory(catalogOpExecutor, metrics_);
pollingFrequencyInSec_ = pollingFrequencyInSec;
initMetrics();
}
@@ -292,6 +358,8 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
}
}
public DeleteEventLog getDeleteEventLog() { return deleteEventLog_; }
/**
* Returns the list of Metastore configurations to validate depending on the hive
* version
@@ -312,9 +380,16 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
(Gauge<String>) () -> getStatus().toString());
metrics_.addGauge(LAST_SYNCED_ID_METRIC,
(Gauge<Long>) () -> lastSyncedEventId_.get());
metrics_.addCounter(NUMBER_OF_SELF_EVENTS);
metrics_.addCounter(NUMBER_OF_TABLE_REFRESHES);
metrics_.addCounter(NUMBER_OF_PARTITION_REFRESHES);
metrics_.addCounter(NUMBER_OF_TABLES_ADDED);
metrics_.addCounter(NUMBER_OF_TABLES_REMOVED);
metrics_.addCounter(NUMBER_OF_DATABASES_ADDED);
metrics_.addCounter(NUMBER_OF_DATABASES_REMOVED);
metrics_.addCounter(NUMBER_OF_PARTITIONS_ADDED);
metrics_.addCounter(NUMBER_OF_PARTITIONS_REMOVED);
metrics_
.addGauge(DELETE_EVENT_LOG_SIZE, (Gauge<Integer>) deleteEventLog_::size);
}
/**
@@ -380,7 +455,11 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
*/
@Override
public synchronized void pause() {
Preconditions.checkState(eventProcessorStatus_ != EventProcessorStatus.PAUSED);
// when concurrent invalidate metadata are running, it is possible the we receive
// a pause method call on a already paused events processor.
if (eventProcessorStatus_ == EventProcessorStatus.PAUSED) {
return;
}
updateStatus(EventProcessorStatus.PAUSED);
LOG.info(String.format("Event processing is paused. Last synced event id is %d",
lastSyncedEventId_.get()));
@@ -405,9 +484,17 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
@Override
public synchronized void start(long fromEventId) {
Preconditions.checkArgument(fromEventId >= 0);
Preconditions.checkState(eventProcessorStatus_ != EventProcessorStatus.ACTIVE,
"Event processing start called when it is already active");
EventProcessorStatus currentStatus = eventProcessorStatus_;
long prevLastSyncedEventId = lastSyncedEventId_.get();
if (currentStatus == EventProcessorStatus.ACTIVE) {
// if events processor is already active, we should make sure that the
// start event id provided is not behind the lastSyncedEventId. This could happen
// when there are concurrent invalidate metadata calls. if we detect such a case
// we should return here.
if (prevLastSyncedEventId >= fromEventId) {
return;
}
}
lastSyncedEventId_.set(fromEventId);
updateStatus(EventProcessorStatus.ACTIVE);
LOG.info(String.format(
@@ -455,14 +542,23 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
}
/**
* Fetch the next batch of NotificationEvents from metastore. The default batch size if
* <code>EVENTS_BATCH_SIZE_PER_RPC</code>
* Gets metastore notification events from the given eventId. The returned list of
* NotificationEvents are filtered using the NotificationFilter provided if it is not
* null.
* @param eventId The returned events are all after this given event id.
* @param skipBatching If this is true all the events since eventId are returned.
* If it is false, only EVENTS_BATCH_SIZE_PER_RPC events are
* returned, caller is expected to issue more calls to this method
* to fetch the remaining events.
* @param filter This is a nullable argument. If not null, the events are filtered
* and then returned using this. Otherwise, all the events are returned.
* @return List of NotificationEvents from metastore since eventId.
* @throws MetastoreNotificationFetchException In case of exceptions from HMS.
*/
@VisibleForTesting
protected List<NotificationEvent> getNextMetastoreEvents()
public List<NotificationEvent> getNextMetastoreEvents(final long eventId,
final boolean skipBatching, @Nullable final NotificationFilter filter)
throws MetastoreNotificationFetchException {
final Timer.Context context = metrics_.getTimer(EVENTS_FETCH_DURATION_METRIC).time();
long lastSyncedEventId = lastSyncedEventId_.get();
try (MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
// fetch the current notification event id. We assume that the polling interval
// is small enough that most of these polling operations result in zero new
@@ -473,24 +569,34 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
long currentEventId = currentNotificationEventId.getEventId();
// no new events since we last polled
if (currentEventId <= lastSyncedEventId) {
if (currentEventId <= eventId) {
return Collections.emptyList();
}
int batchSize = skipBatching ? -1 : EVENTS_BATCH_SIZE_PER_RPC;
NotificationEventResponse response = msClient.getHiveClient()
.getNextNotification(lastSyncedEventId, EVENTS_BATCH_SIZE_PER_RPC, null);
.getNextNotification(eventId, batchSize, filter);
LOG.info(String.format("Received %d events. Start event id : %d",
response.getEvents().size(), lastSyncedEventId));
response.getEvents().size(), eventId));
return response.getEvents();
} catch (TException e) {
throw new MetastoreNotificationFetchException(
"Unable to fetch notifications from metastore. Last synced event id is "
+ lastSyncedEventId, e);
+ eventId, e);
} finally {
context.stop();
}
}
/**
* Fetch the next batch of NotificationEvents from metastore. The default batch size is
* <code>EVENTS_BATCH_SIZE_PER_RPC</code>
*/
@VisibleForTesting
protected List<NotificationEvent> getNextMetastoreEvents()
throws MetastoreNotificationFetchException {
return getNextMetastoreEvents(lastSyncedEventId_.get(), false, null);
}
/**
* This method issues a request to Hive Metastore if needed, based on the current event
* id in metastore and the last synced event_id. Events are fetched in fixed sized
@@ -613,6 +719,7 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
}
lastProcessedEvent = event.metastoreNotificationEvent_;
event.processIfEnabled();
deleteEventLog_.garbageCollect(event.getEventId());
lastSyncedEventId_.set(event.eventId_);
}
}
@@ -654,8 +761,8 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
* a singleton and should only be created during catalogD initialization time, so that
* the start syncId matches with the catalogD startup time.
*
* @param catalog the CatalogServiceCatalog instance to which this event processing
* belongs
* @param catalogOpExecutor the CatalogOpExecutor instance to which this event
* processor belongs.
* @param startSyncFromId Start event id. Events will be polled starting from this
* event id
* @param eventPollingInterval HMS polling interval in seconds
@@ -663,19 +770,17 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
* instantiated
*/
public static synchronized ExternalEventsProcessor getInstance(
CatalogServiceCatalog catalog, long startSyncFromId, long eventPollingInterval)
throws CatalogException {
if (instance != null) {
return instance;
}
CatalogOpExecutor catalogOpExecutor, long startSyncFromId,
long eventPollingInterval) throws CatalogException {
if (instance != null) return instance;
instance =
new MetastoreEventsProcessor(catalog, startSyncFromId, eventPollingInterval);
new MetastoreEventsProcessor(catalogOpExecutor, startSyncFromId,
eventPollingInterval);
return instance;
}
@VisibleForTesting
public MetastoreEventFactory getMetastoreEventFactory() {
@Override
public MetastoreEventFactory getEventsFactory() {
return metastoreEventFactory_;
}

View File

@@ -86,4 +86,14 @@ public class NoOpEventProcessor implements ExternalEventsProcessor {
public TEventProcessorMetricsSummaryResponse getEventProcessorSummary() {
return DEFAULT_SUMMARY_RESPONSE;
}
@Override
public EventFactory getEventsFactory() {
return hmsEvent -> null;
}
@Override
public DeleteEventLog getDeleteEventLog() {
return new DeleteEventLog();
}
}

View File

@@ -45,6 +45,7 @@ import org.apache.impala.catalog.HdfsTable;
import org.apache.impala.catalog.metastore.HmsApiNameEnum;
import org.apache.impala.catalog.monitor.CatalogMonitor;
import org.apache.impala.common.Metrics;
import org.apache.impala.service.CatalogOpExecutor;
import org.apache.impala.thrift.TCatalogdHmsCacheMetrics;
import org.apache.impala.thrift.TCatalogHmsCacheApiMetrics;
import org.apache.impala.service.BackendConfig;
@@ -122,12 +123,10 @@ public class CatalogMetastoreServer extends ThriftHiveMetastore implements
// a blocking call.
private CompletableFuture<Void> serverHandle_;
// reference to the catalog Service catalog object
private final CatalogServiceCatalog catalog_;
private final CatalogOpExecutor catalogOpExecutor_;
public CatalogMetastoreServer(CatalogServiceCatalog catalogServiceCatalog) {
Preconditions.checkNotNull(catalogServiceCatalog);
catalog_ = catalogServiceCatalog;
public CatalogMetastoreServer(CatalogOpExecutor catalogOpExecutor) {
catalogOpExecutor_ = Preconditions.checkNotNull(catalogOpExecutor);
initMetrics();
}
@@ -289,7 +288,7 @@ public class CatalogMetastoreServer extends ThriftHiveMetastore implements
Preconditions.checkState(!started_.get(), "Metastore server is already started");
LOG.info("Starting the Metastore server at port number {}", portNumber);
CatalogMetastoreServiceHandler handler =
new CatalogMetastoreServiceHandler(catalog_,
new CatalogMetastoreServiceHandler(catalogOpExecutor_,
BackendConfig.INSTANCE.fallbackToHMSOnErrors());
// create a proxy class for the ThriftMetastore.Iface and ICatalogMetastoreServer
// so that all the APIs can be invoked via a TimingInvocationHandler

View File

@@ -50,6 +50,7 @@ import org.apache.impala.common.Metrics;
import org.apache.impala.catalog.MetaStoreClientPool.MetaStoreClient;
import org.apache.impala.compat.MetastoreShim;
import org.apache.impala.service.BackendConfig;
import org.apache.impala.service.CatalogOpExecutor;
import org.apache.thrift.TException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -65,9 +66,9 @@ public class CatalogMetastoreServiceHandler extends MetastoreServiceHandler {
private static final Logger LOG = LoggerFactory
.getLogger(CatalogMetastoreServiceHandler.class);
public CatalogMetastoreServiceHandler(CatalogServiceCatalog catalog,
public CatalogMetastoreServiceHandler(CatalogOpExecutor catalogOpExecutor,
boolean fallBackToHMSOnErrors) {
super(catalog, fallBackToHMSOnErrors);
super(catalogOpExecutor, fallBackToHMSOnErrors);
}
@Override

View File

@@ -27,6 +27,7 @@ import org.apache.hadoop.hive.common.ValidWriteIdList;
import org.apache.hadoop.hive.metastore.AbstractThriftHiveMetastore;
import org.apache.hadoop.hive.metastore.DefaultPartitionExpressionProxy;
import org.apache.hadoop.hive.metastore.IMetaStoreClient;
import org.apache.hadoop.hive.metastore.IMetaStoreClient.NotificationFilter;
import org.apache.hadoop.hive.metastore.PartFilterExprUtil;
import org.apache.hadoop.hive.metastore.PartitionExpressionProxy;
import org.apache.hadoop.hive.metastore.api.AbortTxnRequest;
@@ -151,6 +152,7 @@ import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
import org.apache.hadoop.hive.metastore.api.NoSuchTxnException;
import org.apache.hadoop.hive.metastore.api.NotNullConstraintsRequest;
import org.apache.hadoop.hive.metastore.api.NotNullConstraintsResponse;
import org.apache.hadoop.hive.metastore.api.NotificationEvent;
import org.apache.hadoop.hive.metastore.api.NotificationEventRequest;
import org.apache.hadoop.hive.metastore.api.NotificationEventResponse;
import org.apache.hadoop.hive.metastore.api.NotificationEventsCountRequest;
@@ -210,7 +212,6 @@ import org.apache.hadoop.hive.metastore.api.Table;
import org.apache.hadoop.hive.metastore.api.TableMeta;
import org.apache.hadoop.hive.metastore.api.TableStatsRequest;
import org.apache.hadoop.hive.metastore.api.TableStatsResult;
import org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore.Iface;
import org.apache.hadoop.hive.metastore.api.TruncateTableRequest;
import org.apache.hadoop.hive.metastore.api.TruncateTableResponse;
import org.apache.hadoop.hive.metastore.api.TxnAbortedException;
@@ -262,14 +263,19 @@ import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
import org.apache.impala.catalog.CatalogHmsAPIHelper;
import org.apache.impala.catalog.DatabaseNotFoundException;
import org.apache.impala.catalog.Db;
import org.apache.impala.catalog.CatalogServiceCatalog;
import org.apache.impala.catalog.IncompleteTable;
import org.apache.impala.catalog.MetaStoreClientPool.MetaStoreClient;
import org.apache.impala.catalog.events.MetastoreEvents.DropTableEvent;
import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEventType;
import org.apache.impala.catalog.events.MetastoreEventsProcessor;
import org.apache.impala.common.ImpalaException;
import org.apache.impala.common.ImpalaRuntimeException;
import org.apache.impala.common.Reference;
import org.apache.impala.common.Pair;
import org.apache.impala.compat.MetastoreShim;
import org.apache.impala.service.BackendConfig;
import org.apache.impala.service.CatalogOpExecutor;
import org.apache.impala.util.AcidUtils;
import org.apache.impala.thrift.TTableName;
import org.apache.thrift.TException;
@@ -300,6 +306,7 @@ public abstract class MetastoreServiceHandler extends AbstractThriftHiveMetastor
+ "table %s to the backing HiveMetastore service";
// constant used for logging error messages
protected final CatalogOpExecutor catalogOpExecutor_;
protected final CatalogServiceCatalog catalog_;
protected final boolean fallBackToHMSOnErrors_;
// TODO handle session configuration
@@ -308,9 +315,10 @@ public abstract class MetastoreServiceHandler extends AbstractThriftHiveMetastor
protected final String defaultCatalogName_;
protected final boolean invalidateCacheOnDDLs_;
public MetastoreServiceHandler(CatalogServiceCatalog catalog,
public MetastoreServiceHandler(CatalogOpExecutor catalogOpExecutor,
boolean fallBackToHMSOnErrors) {
catalog_ = Preconditions.checkNotNull(catalog);
catalogOpExecutor_ = Preconditions.checkNotNull(catalogOpExecutor);
catalog_ = Preconditions.checkNotNull(catalogOpExecutor.getCatalog());
fallBackToHMSOnErrors_ = fallBackToHMSOnErrors;
LOG.info("Fallback to hive metastore service on errors is {}",
fallBackToHMSOnErrors_);
@@ -639,8 +647,9 @@ public abstract class MetastoreServiceHandler extends AbstractThriftHiveMetastor
public void drop_table(String dbname, String tblname, boolean deleteData)
throws NoSuchObjectException, MetaException, TException {
try (MetaStoreClient client = catalog_.getMetaStoreClient()) {
long eventId = getCurrentEventId(client);
client.getHiveClient().getThriftClient().drop_table(dbname, tblname, deleteData);
removeNonTransactionalTableIfExists(dbname, tblname, "drop_table");
removeNonTransactionalTableIfExists(eventId, dbname, tblname, "drop_table");
}
}
@@ -650,10 +659,11 @@ public abstract class MetastoreServiceHandler extends AbstractThriftHiveMetastor
EnvironmentContext environmentContext)
throws NoSuchObjectException, MetaException, TException {
try (MetaStoreClient client = catalog_.getMetaStoreClient()) {
long eventId = getCurrentEventId(client);
client.getHiveClient().getThriftClient()
.drop_table_with_environment_context(dbname, tblname, deleteData,
environmentContext);
removeNonTransactionalTableIfExists(dbname, tblname,
removeNonTransactionalTableIfExists(eventId, dbname, tblname,
"drop_table_with_environment_context");
}
}
@@ -2917,12 +2927,19 @@ public abstract class MetastoreServiceHandler extends AbstractThriftHiveMetastor
return;
}
/**
* Gets the current event id from the hive metastore.
*/
private long getCurrentEventId(MetaStoreClient msClient) throws TException {
return msClient.getHiveClient().getCurrentNotificationEventId().getEventId();
}
/**
* This method is identical to invalidateNonTransactionalTableIfExists()
* except that it removes(and not invalidates) table from the cache on
* ddls like drop_table
*/
private void removeNonTransactionalTableIfExists(String dbNameWithCatalog,
private void removeNonTransactionalTableIfExists(long eventId, String dbNameWithCatalog,
String tableName, String apiName) throws MetaException {
// return immediately if flag invalidateCacheOnDDLs_ is false
if (!invalidateCacheOnDDLs_) {
@@ -2932,9 +2949,12 @@ public abstract class MetastoreServiceHandler extends AbstractThriftHiveMetastor
return;
}
// Parse db name. Throw error if parsing fails.
String dbName = dbNameWithCatalog;
String dbName;
String catName;
try {
dbName = MetaStoreUtils.parseDbName(dbNameWithCatalog, serverConf_)[1];
String[] catAndDbName = MetaStoreUtils.parseDbName(dbNameWithCatalog, serverConf_);
catName = catAndDbName[0];
dbName = catAndDbName[1];
} catch (MetaException ex) {
LOG.error("Successfully executed HMS api: {} but encountered error " +
"when parsing dbName {} to invalidate/remove table from cache " +
@@ -2942,47 +2962,33 @@ public abstract class MetastoreServiceHandler extends AbstractThriftHiveMetastor
ex.getMessage());
throw ex;
}
org.apache.impala.catalog.Table catalogTbl = null;
try {
catalogTbl = catalog_.getTable(dbName, tableName);
} catch (DatabaseNotFoundException ex) {
LOG.debug(ex.getMessage());
return;
}
if (catalogTbl == null) {
LOG.debug("{}.{} does not exist", dbName, tableName);
return;
}
if (catalogTbl instanceof IncompleteTable) {
LOG.debug("Removing incomplete table {} from cache " +
"due to HMS API: ", catalogTbl.getFullName(), apiName);
if (catalog_.removeTable(dbName, tableName) != null) {
LOG.info("Removed incomplete table {} from cache due " +
"to HMS API: ", catalogTbl.getFullName(), apiName);
List<NotificationEvent> events = MetastoreEventsProcessor
.getNextMetastoreEvents(catalog_, eventId,
event -> event.getEventType()
.equalsIgnoreCase(DropTableEvent.DROP_TABLE_EVENT_TYPE)
&& catName.equalsIgnoreCase(event.getCatName())
&& dbName.equalsIgnoreCase(event.getDbName())
&& tableName.equalsIgnoreCase(event.getTableName()));
if (events.isEmpty()) {
throw new MetaException(
"Drop table event not received. Check if notification events are "
+ "configured in hive metastore");
}
return;
long dropEventId = events.get(events.size() - 1).getEventId();
Reference<Boolean> tblAddedLater = new Reference<>();
boolean removedTbl = catalogOpExecutor_
.removeTableIfNotAddedLater(dropEventId, dbName, tableName, tblAddedLater);
if (removedTbl) {
LOG.info("Removed non transactional table {}.{} from catalogd cache due to " +
"HMS api: {}", dbName, tableName, apiName);
}
} catch (ImpalaException e) {
String msg =
"Unable to process the DROP table event for table " + dbName + "." + tableName;
LOG.error(msg, e);
throw new MetaException(msg);
}
Map<String, String> tblProperties = catalogTbl.getMetaStoreTable().getParameters();
if (tblProperties == null || MetaStoreUtils.isTransactionalTable(tblProperties)) {
LOG.debug("Table {} is transactional. " +
"Not removing it from catalogd cache", catalogTbl.getFullName());
return;
}
LOG.debug("Removing non transactional table {} due to HMS api {}",
catalogTbl.getFullName(), apiName);
Reference<Boolean> tableFound = new Reference<>();
Reference<Boolean> tableMatched = new Reference<>();
// TODO: Move method removeTableIfExists to CatalogOpExecutor
// as suggested in
// IMPALA-10502 (patch: https://gerrit.cloudera.org/#/c/17308/)
org.apache.impala.catalog.Table removedTable =
catalog_.removeTableIfExists(catalogTbl.getMetaStoreTable(),
tableFound, tableMatched);
if (removedTable != null) {
LOG.info("Removed non transactional table {} from catalogd cache due to " +
"HMS api: {}", catalogTbl.getFullName(), apiName);
}
return;
}
/*

View File

@@ -17,6 +17,7 @@
package org.apache.impala.service;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
@@ -25,6 +26,7 @@ import java.util.ArrayList;
import java.util.List;
import java.util.UUID;
import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
import org.apache.impala.authorization.AuthorizationConfig;
import org.apache.impala.authorization.AuthorizationFactory;
import org.apache.impala.authorization.AuthorizationManager;
@@ -33,6 +35,14 @@ import org.apache.impala.catalog.CatalogServiceCatalog;
import org.apache.impala.catalog.Db;
import org.apache.impala.catalog.FeDb;
import org.apache.impala.catalog.Function;
import org.apache.impala.catalog.MetaStoreClientPool;
import org.apache.impala.catalog.MetaStoreClientPool.MetaStoreClient;
import org.apache.impala.catalog.events.ExternalEventsProcessor;
import org.apache.impala.catalog.events.MetastoreEventsProcessor;
import org.apache.impala.catalog.events.NoOpEventProcessor;
import org.apache.impala.catalog.metastore.CatalogMetastoreServer;
import org.apache.impala.catalog.metastore.ICatalogMetastoreServer;
import org.apache.impala.catalog.metastore.NoOpCatalogMetastoreServer;
import org.apache.impala.catalog.monitor.CatalogMonitor;
import org.apache.impala.catalog.monitor.CatalogOperationMetrics;
import org.apache.impala.compat.MetastoreShim;
@@ -85,6 +95,7 @@ public class JniCatalog {
new TBinaryProtocol.Factory();
private final CatalogServiceCatalog catalog_;
private final CatalogOpExecutor catalogOpExecutor_;
private final ICatalogMetastoreServer catalogMetastoreServer_;
private final AuthorizationManager authzManager_;
// A unique identifier for this instance of the Catalog Service.
@@ -128,17 +139,72 @@ public class JniCatalog {
MetastoreShim.setHiveClientCapabilities();
}
MetaStoreClientPool metaStoreClientPool = new MetaStoreClientPool(
CatalogServiceCatalog.INITIAL_META_STORE_CLIENT_POOL_SIZE,
cfg.initial_hms_cnxn_timeout_s);
catalog_ = new CatalogServiceCatalog(cfg.load_catalog_in_background,
cfg.num_metadata_loading_threads, cfg.initial_hms_cnxn_timeout_s, getServiceId(),
cfg.local_library_path);
cfg.num_metadata_loading_threads, getServiceId(),
cfg.local_library_path, metaStoreClientPool);
authzManager_ = authzFactory.newAuthorizationManager(catalog_);
catalog_.setAuthzManager(authzManager_);
catalogOpExecutor_ = new CatalogOpExecutor(catalog_, authzConfig, authzManager_);
ExternalEventsProcessor eventsProcessor = getEventsProcessor(metaStoreClientPool,
catalogOpExecutor_);
catalog_.setMetastoreEventProcessor(eventsProcessor);
catalog_.startEventsProcessor();
catalogMetastoreServer_ = getCatalogMetastoreServer(catalogOpExecutor_);
catalog_.setCatalogMetastoreServer(catalogMetastoreServer_);
catalogMetastoreServer_.start();
try {
catalog_.reset();
} catch (CatalogException e) {
LOG.error("Error initializing Catalog. Please run 'invalidate metadata'", e);
}
catalogOpExecutor_ = new CatalogOpExecutor(catalog_, authzConfig, authzManager_);
}
/**
* Returns an instance of CatalogMetastoreServer if start_hms_server configuration is
* true. Otherwise, returns a NoOpCatalogMetastoreServer
*/
@VisibleForTesting
private ICatalogMetastoreServer getCatalogMetastoreServer(
CatalogOpExecutor catalogOpExecutor) {
if (!BackendConfig.INSTANCE.startHmsServer()) {
return NoOpCatalogMetastoreServer.INSTANCE;
}
int portNumber = BackendConfig.INSTANCE.getHMSPort();
Preconditions.checkState(portNumber > 0, "Invalid port number for HMS service.");
return new CatalogMetastoreServer(catalogOpExecutor);
}
/**
* Returns a Metastore event processor object if
* <code>BackendConfig#getHMSPollingIntervalInSeconds</code> returns a non-zero
*.value of polling interval. Otherwise, returns a no-op events processor. It is
* important to fetch the current notification event id at the Catalog service
* initialization time so that event processor starts to sync at the event id
* corresponding to the catalog start time.
*/
private ExternalEventsProcessor getEventsProcessor(
MetaStoreClientPool metaStoreClientPool, CatalogOpExecutor catalogOpExecutor)
throws ImpalaException {
long eventPollingInterval = BackendConfig.INSTANCE.getHMSPollingIntervalInSeconds();
if (eventPollingInterval <= 0) {
LOG.info(String
.format("Metastore event processing is disabled. Event polling interval is %d",
eventPollingInterval));
return NoOpEventProcessor.getInstance();
}
try (MetaStoreClient metaStoreClient = metaStoreClientPool.getClient()) {
CurrentNotificationEventId currentNotificationId =
metaStoreClient.getHiveClient().getCurrentNotificationEventId();
return MetastoreEventsProcessor.getInstance(
catalogOpExecutor, currentNotificationId.getEventId(), eventPollingInterval);
} catch (TException e) {
LOG.error("Unable to fetch the current notification event id from metastore.", e);
throw new CatalogException(
"Fatal error while initializing metastore event processor", e);
}
}
public static TUniqueId getServiceId() { return catalogServiceId_; }

View File

@@ -23,10 +23,13 @@ import java.util.Random;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId;
import org.apache.impala.authorization.NoopAuthorizationFactory;
import org.apache.impala.authorization.NoopAuthorizationFactory.NoopAuthorizationManager;
import org.apache.impala.catalog.CatalogException;
import org.apache.impala.catalog.CatalogServiceCatalog;
import org.apache.impala.catalog.MetaStoreClientPool.MetaStoreClient;
import org.apache.impala.catalog.events.MetastoreEventsProcessor.EventProcessorStatus;
import org.apache.impala.service.CatalogOpExecutor;
import org.apache.impala.util.RandomHiveQueryRunner;
import org.apache.impala.common.Pair;
import org.apache.impala.compat.MetastoreShim;
@@ -104,11 +107,14 @@ public class EventsProcessorStressTest {
@BeforeClass
public static void setupTestEnv() throws Exception {
catalog_ = CatalogServiceTestCatalog.create();
CatalogOpExecutor catalogOpExecutor = new CatalogOpExecutor(catalog_,
new NoopAuthorizationFactory().getAuthorizationConfig(),
new NoopAuthorizationManager());
try (MetaStoreClient metaStoreClient = catalog_.getMetaStoreClient()) {
CurrentNotificationEventId currentNotificationId =
metaStoreClient.getHiveClient().getCurrentNotificationEventId();
eventsProcessor_ = new SynchronousHMSEventProcessorForTests(
catalog_, currentNotificationId.getEventId(), 10L);
catalogOpExecutor, currentNotificationId.getEventId(), 10L);
eventsProcessor_.start();
}
catalog_.setMetastoreEventProcessor(eventsProcessor_);

View File

@@ -64,11 +64,11 @@ import org.apache.hadoop.hive.metastore.api.SerDeInfo;
import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
import org.apache.hadoop.hive.metastore.api.PrincipalType;
import org.apache.hadoop.hive.metastore.messaging.AlterTableMessage;
import org.apache.hadoop.hive.metastore.messaging.CommitTxnMessage;
import org.apache.hadoop.hive.metastore.messaging.InsertMessage;
import org.apache.hadoop.io.IOUtils;
import org.apache.impala.analysis.FunctionName;
import org.apache.impala.analysis.HdfsUri;
import org.apache.impala.authorization.AuthorizationConfig;
import org.apache.impala.authorization.AuthorizationManager;
import org.apache.impala.authorization.NoopAuthorizationFactory;
import org.apache.impala.authorization.NoopAuthorizationFactory.NoopAuthorizationManager;
import org.apache.impala.catalog.CatalogException;
@@ -88,7 +88,6 @@ import org.apache.impala.catalog.Table;
import org.apache.impala.catalog.Type;
import org.apache.impala.catalog.events.ConfigValidator.ValidationResult;
import org.apache.impala.catalog.events.MetastoreEvents.AlterTableEvent;
import org.apache.impala.catalog.events.MetastoreEvents.InsertEvent;
import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEvent;
import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEventPropertyKey;
import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEventType;
@@ -194,7 +193,7 @@ public class MetastoreEventsProcessorTest {
CurrentNotificationEventId currentNotificationId =
metaStoreClient.getHiveClient().getCurrentNotificationEventId();
eventsProcessor_ = new SynchronousHMSEventProcessorForTests(
catalog_, currentNotificationId.getEventId(), 10L);
catalogOpExecutor_, currentNotificationId.getEventId(), 10L);
eventsProcessor_.start();
}
catalog_.setMetastoreEventProcessor(eventsProcessor_);
@@ -435,45 +434,6 @@ public class MetastoreEventsProcessorTest {
assertNotNull(catalog_.getDb(TEST_DB_NAME));
}
/**
* Test to verify that DROP_DATABASE event is processed such that it removes the DB from
* Catalog only if the CREATION_TIME of the Catalog's DB object is less than or equal to
* that in the event.
*/
@Test
public void testDropDatabaseCreationTime()
throws ImpalaException, InterruptedException {
long filteredCount = eventsProcessor_.getMetrics()
.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).getCount();
createDatabaseFromImpala(TEST_DB_NAME, "Test DB for CREATION_TIME");
// now drop the database with cascade option
dropDatabaseCascadeFromImpala(TEST_DB_NAME);
// Adding sleep here to make sure that the CREATION_TIME is not same
// as the previous CREATE_DB operation, so as to trigger the filtering logic
// based on CREATION_TIME in DROP_DB event processing. This is currently a
// limitation : the DROP_DB event filtering expects that while processing events,
// the CREATION_TIME of two Databases with same name won't have the same
// creation timestamp.
sleep(2000);
// Create database again with same name
createDatabaseFromImpala(TEST_DB_NAME, "Test DB for CREATION_TIME");
eventsProcessor_.processEvents();
// Here, we expect the events CREATE_DB, DROP_DB, CREATE_DB for the
// same Database name. Hence, the DROP_DB event should not be processed,
// as the CREATION_TIME of the catalog's Database object should be greater
// than that in the DROP_DB notification event. Two events are filtered here,
// 1 : first CREATE_DATABASE as it is followed by another create of the same name.
// 2 : DROP_DATABASE as it is trying to drop a database which is again created.
assertEquals(filteredCount + 2, eventsProcessor_.getMetrics()
.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC)
.getCount());
// Teardown step - Drop the created DB
dropDatabaseCascadeFromImpala(TEST_DB_NAME);
}
@Test
public void testAlterDatabaseEvents() throws TException, ImpalaException {
createDatabase(TEST_DB_NAME, null);
@@ -531,9 +491,10 @@ public class MetastoreEventsProcessorTest {
createDatabaseFromImpala(TEST_DB_NAME, null);
assertNotNull("Db should have been found after create database statement",
catalog_.getDb(TEST_DB_NAME));
eventsProcessor_.processEvents();
long numberOfSelfEventsBefore =
eventsProcessor_.getMetrics()
.getCounter(MetastoreEventsProcessor.NUMBER_OF_SELF_EVENTS)
.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC)
.getCount();
String owner = catalog_.getDb(TEST_DB_NAME).getMetaStoreDb().getOwnerName();
String newOwnerUser = "newUserFromImpala";
@@ -551,7 +512,7 @@ public class MetastoreEventsProcessorTest {
long selfEventsCountAfter =
eventsProcessor_.getMetrics()
.getCounter(MetastoreEventsProcessor.NUMBER_OF_SELF_EVENTS)
.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC)
.getCount();
// 2 alter commands above, so we expect the count to go up by 2
assertEquals("Unexpected number of self-events generated",
@@ -567,9 +528,10 @@ public class MetastoreEventsProcessorTest {
createDatabaseFromImpala(TEST_DB_NAME, null);
assertNotNull("Db should have been found after create database statement",
catalog_.getDb(TEST_DB_NAME));
eventsProcessor_.processEvents();
long numberOfSelfEventsBefore =
eventsProcessor_.getMetrics()
.getCounter(MetastoreEventsProcessor.NUMBER_OF_SELF_EVENTS).getCount();
.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).getCount();
// Create a dummy scalar function.
String fnName = "fn1";
@@ -586,7 +548,7 @@ public class MetastoreEventsProcessorTest {
eventsProcessor_.processEvents();
long numberOfSelfEventsAfter =
eventsProcessor_.getMetrics()
.getCounter(MetastoreEventsProcessor.NUMBER_OF_SELF_EVENTS).getCount();
.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).getCount();
assertEquals("Unexpected number of self-events generated",
numberOfSelfEventsBefore + 2, numberOfSelfEventsAfter);
@@ -628,6 +590,17 @@ public class MetastoreEventsProcessorTest {
assertTrue("Newly created table should be instance of IncompleteTable",
catalog_.getTable(TEST_DB_NAME, testPartitionedTbl)
instanceof IncompleteTable);
// Test create table on a drop database event.
dropDatabaseCascadeFromImpala(TEST_DB_NAME);
assertNull("Database not expected to exist.", catalog_.getDb(TEST_DB_NAME));
createDatabaseFromImpala(TEST_DB_NAME, null);
eventsProcessor_.processEvents();
createTable("createondroppeddb", false);
dropDatabaseCascadeFromImpala(TEST_DB_NAME);
eventsProcessor_.processEvents();
assertEquals(EventProcessorStatus.ACTIVE, eventsProcessor_.getStatus());
}
/**
@@ -738,11 +711,12 @@ public class MetastoreEventsProcessorTest {
String tableToInsertPart = "tbl_with_mul_part";
String tableToInsertMulPart = "tbl_to_insert_mul_part";
createInsertTestTbls(tableToInsertPart, tableToInsertMulPart);
eventsProcessor_.processEvents();
// count self event from here, numberOfSelfEventsBefore=4 as we have 4 ADD PARTITION
// events
long numberOfSelfEventsBefore =
eventsProcessor_.getMetrics()
.getCounter(MetastoreEventsProcessor.NUMBER_OF_SELF_EVENTS)
.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC)
.getCount();
runInsertTest(tableToInsertPart, tableToInsertMulPart, numberOfSelfEventsBefore,
false);
@@ -757,11 +731,12 @@ public class MetastoreEventsProcessorTest {
String tableToInsertPart = "tbl_with_mul_part";
String tableToInsertMulPart = "tbl_to_insert_mul_part";
createInsertTestTbls(tableToInsertPart, tableToInsertMulPart);
eventsProcessor_.processEvents();
// count self event from here, numberOfSelfEventsBefore=4 as we have 4 ADD PARTITION
// events
long numberOfSelfEventsBefore =
eventsProcessor_.getMetrics()
.getCounter(MetastoreEventsProcessor.NUMBER_OF_SELF_EVENTS)
.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC)
.getCount();
runInsertTest(tableToInsertPart, tableToInsertMulPart, numberOfSelfEventsBefore,
true);
@@ -858,7 +833,7 @@ public class MetastoreEventsProcessorTest {
updated_partitions.put(partition2, updatedPartition2);
insertMulPartFromImpala(tableToInsertMulPart, tableToInsertPart, updated_partitions,
overwrite);
// we expect 3 INSERT events (2 for the insertTbl and 1 for multiInsertTbl)
// we expect 4 INSERT events (2 for the insertTbl and 2 for multiInsertTbl)
List<NotificationEvent> events = eventsProcessor_.getNextMetastoreEvents();
assertEquals(4, events.size());
assertEquals(tbl1Part1Files, getFilesFromEvent(events.get(0)));
@@ -869,6 +844,7 @@ public class MetastoreEventsProcessorTest {
// Test insert into table
String unpartitionedTbl = "tbl_to_insert";
// create table self-event 5
createTableLike("functional", "tinytable", TEST_DB_NAME, unpartitionedTbl);
HdfsTable tinyTable = (HdfsTable) catalog_
.getOrLoadTable("functional", "tinytable", "test", null);
@@ -878,17 +854,18 @@ public class MetastoreEventsProcessorTest {
copyFiles(tinyTable.getFileSystem(), new Path(tinyTable.getHdfsBaseDir()),
unpartTable.getFileSystem(), new Path(unpartTable.getHdfsBaseDir()),
overwrite, "copy_");
// insert self-event 6
insertFromImpala(unpartitionedTbl, false, "", "", overwrite, copied_files);
eventsProcessor_.processEvents();
long selfEventsCountAfter =
eventsProcessor_.getMetrics()
.getCounter(MetastoreEventsProcessor.NUMBER_OF_SELF_EVENTS)
.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC)
.getCount();
// 2 single insert partition events, 1 multi insert partitions which includes 2 single
// insert events 1 single insert table event
// insert events 1 single insert table event, 1 create table event
assertEquals("Unexpected number of self-events generated",
numberOfSelfEventsBefore + 5, selfEventsCountAfter);
numberOfSelfEventsBefore + 6, selfEventsCountAfter);
}
private List<String> getFilesFromEvent(NotificationEvent event) {
@@ -1119,39 +1096,39 @@ public class MetastoreEventsProcessorTest {
// clean up
dropDatabaseCascadeFromImpala("new_db");
// check invalidate after alter table add parameter
// check refresh after alter table add parameter
loadTable(testTblName);
alterTableAddParameter(testTblName, "somekey", "someval");
eventsProcessor_.processEvents();
assertFalse("Table should have been refreshed after alter table add parameter",
catalog_.getTable(TEST_DB_NAME, testTblName)
instanceof IncompleteTable);
// check invalidate after alter table add col
// check refresh after alter table add col
loadTable(testTblName);
alterTableAddCol(testTblName, "newCol", "int", "null");
eventsProcessor_.processEvents();
assertFalse("Table should have been refreshed after alter table add column",
catalog_.getTable(TEST_DB_NAME, testTblName)
instanceof IncompleteTable);
// check invalidate after alter table change column type
// check refresh after alter table change column type
loadTable(testTblName);
altertableChangeCol(testTblName, "newCol", "string", null);
eventsProcessor_.processEvents();
assertFalse("Table should have been refreshed after changing column type",
catalog_.getTable(TEST_DB_NAME, testTblName)
instanceof IncompleteTable);
// check invalidate after alter table remove column
// check refresh after alter table remove column
loadTable(testTblName);
alterTableRemoveCol(testTblName, "newCol");
eventsProcessor_.processEvents();
assertFalse("Table should have been refreshed after removing a column",
catalog_.getTable(TEST_DB_NAME, testTblName)
instanceof IncompleteTable);
// 5 alters above. Each one of them except rename should increment the counter by 1
long numberOfInvalidatesAfter = eventsProcessor_.getMetrics()
// 4 alters above. Each one of them except rename should increment the counter by 1
long numOfRefreshesAfter = eventsProcessor_.getMetrics()
.getCounter(MetastoreEventsProcessor.NUMBER_OF_TABLE_REFRESHES).getCount();
assertEquals("Unexpected number of table refreshes",
numOfRefreshesBefore + 4, numberOfInvalidatesAfter);
numOfRefreshesBefore + 4, numOfRefreshesAfter);
// Check if trivial alters are ignored.
loadTable(testTblName);
alterTableChangeTrivialProperties(testTblName);
@@ -1277,9 +1254,9 @@ public class MetastoreEventsProcessorTest {
private static class HMSFetchNotificationsEventProcessor
extends MetastoreEventsProcessor {
HMSFetchNotificationsEventProcessor(
CatalogServiceCatalog catalog, long startSyncFromId, long pollingFrequencyInSec)
CatalogOpExecutor catalogOp, long startSyncFromId, long pollingFrequencyInSec)
throws CatalogException {
super(catalog, startSyncFromId, pollingFrequencyInSec);
super(catalogOp, startSyncFromId, pollingFrequencyInSec);
}
@Override
@@ -1298,9 +1275,13 @@ public class MetastoreEventsProcessorTest {
* Tests event processor is active after HMS restarts.
*/
@Test
public void testEventProcessorFetchAfterHMSRestart() throws CatalogException {
public void testEventProcessorFetchAfterHMSRestart() throws ImpalaException {
CatalogServiceCatalog catalog = CatalogServiceTestCatalog.create();
CatalogOpExecutor catalogOpExecutor = new CatalogOpExecutor(catalog,
new NoopAuthorizationFactory().getAuthorizationConfig(),
new NoopAuthorizationManager());
MetastoreEventsProcessor fetchProcessor =
new HMSFetchNotificationsEventProcessor(CatalogServiceTestCatalog.create(),
new HMSFetchNotificationsEventProcessor(catalogOpExecutor,
eventsProcessor_.getCurrentEventId(), 2L);
fetchProcessor.start();
try {
@@ -1432,71 +1413,6 @@ public class MetastoreEventsProcessorTest {
assertNull(catalog_.getTable(TEST_DB_NAME, testTblName));
}
/**
* Creates events like create, drop with the same tblName. In such case the create
* table should not create a in
*/
@Test
public void testEventFiltering() throws Exception {
createDatabaseFromImpala(TEST_DB_NAME, "");
final String testTblName = "testEventFiltering";
createTableFromImpala(TEST_DB_NAME, testTblName, false);
loadTable(testTblName);
assertNotNull(catalog_.getTable(TEST_DB_NAME, testTblName));
dropTableFromImpala(TEST_DB_NAME, testTblName);
// the create table event should be filtered out
verifyFilterEvents(3, 2, Arrays.asList(CREATE_DATABASE, DROP_TABLE));
// test the table rename case
createTableFromImpala(TEST_DB_NAME, testTblName, false);
renameTableFromImpala(testTblName, "new_name");
// create table gets filtered out since it was renamed immediated after
verifyFilterEvents(2, 1, Arrays.asList(ALTER_TABLE));
//cleanup
dropDatabaseCascadeFromImpala(TEST_DB_NAME);
eventsProcessor_.processEvents();
// test when multiple events can be filtered out
// create_db, create_tbl, drop_tbl, drop_db
createDatabaseFromImpala(TEST_DB_NAME, "desc");
createTableFromImpala(TEST_DB_NAME, testTblName, false);
loadTable(TEST_DB_NAME, testTblName);
assertNotNull(catalog_.getTable(TEST_DB_NAME, testTblName));
dropTableFromImpala(TEST_DB_NAME, testTblName);
dropDatabaseCascadeFromImpala(TEST_DB_NAME);
verifyFilterEvents(4, 2, Arrays.asList(DROP_TABLE, DROP_DATABASE));
// create event stream s.t inverse events have gaps from their counterparts
createDatabase(TEST_DB_NAME, null);
// unrelated event
createTable("dummy", false);
createTable(testTblName, false);
// dummy events
alterTableAddParameter(testTblName, "paramkey", "paramVal");
alterTableAddParameter(testTblName, "paramkey1", "paramVal2");
dropTable(testTblName);
// this would generate drop_table for dummy table as well
dropDatabaseCascade(TEST_DB_NAME);
verifyFilterEvents(8, 5, Arrays.asList(ALTER_TABLE, ALTER_TABLE, DROP_TABLE,
DROP_TABLE, DROP_DATABASE));
}
private void verifyFilterEvents(int total, int numFiltered,
List<MetastoreEventType> expectedFilteredEventTypes) throws ImpalaException {
List<NotificationEvent> events = eventsProcessor_.getNextMetastoreEvents();
assertEquals(total, events.size());
List<MetastoreEvent> filteredEvents =
eventsProcessor_.getMetastoreEventFactory().getFilteredEvents(events);
assertEquals(numFiltered, filteredEvents.size());
int i = 0;
for (MetastoreEvent e : filteredEvents) {
assertEquals(expectedFilteredEventTypes.get(i++), e.eventType_);
}
eventsProcessor_.processEvents();
assertEquals(EventProcessorStatus.ACTIVE, eventsProcessor_.getStatus());
}
/**
* Similar to create,drop,create sequence table as in
* <code>testCreateDropCreateTableFromImpala</code> but operates on Database instead
@@ -1640,6 +1556,23 @@ public class MetastoreEventsProcessorTest {
}
}
private static class FakeCatalogOpExecutorForTests extends CatalogOpExecutor {
public FakeCatalogOpExecutorForTests(CatalogServiceCatalog catalog,
AuthorizationConfig authzConfig,
AuthorizationManager authzManager)
throws ImpalaException {
super(catalog, authzConfig, authzManager);
}
public static CatalogOpExecutor create() throws ImpalaException {
return new FakeCatalogOpExecutorForTests(
FakeCatalogServiceCatalogForFlagTests.create(),
new NoopAuthorizationFactory().getAuthorizationConfig(),
new NoopAuthorizationManager());
}
}
/**
* Test catalog service catalog which takes a value of db and tbl flags for a given
* table
@@ -1666,6 +1599,7 @@ public class MetastoreEventsProcessorTest {
cs = new FakeCatalogServiceCatalogForFlagTests(false, 16, new TUniqueId(),
System.getProperty("java.io.tmpdir"), new MetaStoreClientPool(0, 0));
cs.setAuthzManager(new NoopAuthorizationManager());
cs.setMetastoreEventProcessor(NoOpEventProcessor.getInstance());
cs.reset();
} catch (ImpalaException e) {
throw new IllegalStateException(e.getMessage(), e);
@@ -1740,14 +1674,16 @@ public class MetastoreEventsProcessorTest {
dbParams.put(MetastoreEventPropertyKey.DISABLE_EVENT_HMS_SYNC.getKey(), dbFlag);
}
CatalogServiceCatalog fakeCatalog = FakeCatalogServiceCatalogForFlagTests.create();
((FakeCatalogServiceCatalogForFlagTests) fakeCatalog)
.setFlags(dbName, tblName, dbFlag, tblFlagTransition.first);
CatalogOpExecutor fakeCatalogOpExecutor = FakeCatalogOpExecutorForTests.create();
FakeCatalogServiceCatalogForFlagTests fakeCatalog =
(FakeCatalogServiceCatalogForFlagTests) fakeCatalogOpExecutor
.getCatalog();
fakeCatalog.setFlags(dbName, tblName, dbFlag, tblFlagTransition.first);
NotificationEvent fakeAlterTableNotification =
createFakeAlterTableNotification(dbName, tblName, tableBefore, tableAfter);
AlterTableEvent alterTableEvent = new AlterTableEvent(
fakeCatalog, eventsProcessor_.getMetrics(), fakeAlterTableNotification);
fakeCatalogOpExecutor, eventsProcessor_.getMetrics(), fakeAlterTableNotification);
Assert.assertFalse("Alter table which changes the flags should not be skipped. "
+ printFlagTransistions(dbFlag, tblFlagTransition),
alterTableEvent.isEventProcessingDisabled());
@@ -1759,7 +1695,8 @@ public class MetastoreEventsProcessorTest {
NotificationEvent nextNotification =
createFakeAlterTableNotification(dbName, tblName, tableAfter, nextTable);
alterTableEvent =
new AlterTableEvent(fakeCatalog, eventsProcessor_.getMetrics(), nextNotification);
new AlterTableEvent(fakeCatalogOpExecutor, eventsProcessor_.getMetrics(),
nextNotification);
if (shouldNextEventBeSkipped) {
assertTrue("Alter table event should not skipped following this table flag "
+ "transition. " + printFlagTransistions(dbFlag, tblFlagTransition),
@@ -2188,8 +2125,9 @@ public class MetastoreEventsProcessorTest {
eventsProcessor_.processEvents();
final String testTblName = "testSelfEventsForTable";
createTableFromImpala(TEST_DB_NAME, testTblName, true);
eventsProcessor_.processEvents();
long numberOfSelfEventsBefore = eventsProcessor_.getMetrics()
.getCounter(MetastoreEventsProcessor.NUMBER_OF_SELF_EVENTS).getCount();
.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).getCount();
alterTableSetTblPropertiesFromImpala(testTblName);
eventsProcessor_.processEvents();
@@ -2216,11 +2154,6 @@ public class MetastoreEventsProcessorTest {
partitionDef.addToPartition_spec(new TPartitionKeyValue("p1", "100"));
partitionDef.addToPartition_spec(new TPartitionKeyValue("p2", "200"));
// add partition
alterTableAddPartition(TEST_DB_NAME, testTblName, partitionDef);
eventsProcessor_.processEvents();
confirmTableIsLoaded(TEST_DB_NAME, testTblName);
// set fileformat
alterTableSetFileFormatFromImpala(
TEST_DB_NAME, testTblName, THdfsFileFormat.TEXT);
@@ -2261,9 +2194,8 @@ public class MetastoreEventsProcessorTest {
//add test for alterCommentOnTableOrView
long selfEventsCountAfter = eventsProcessor_.getMetrics()
.getCounter(MetastoreEventsProcessor.NUMBER_OF_SELF_EVENTS).getCount();
// 10 alter commands above. Everyone except alterRename should generate
// self-events so we expect the count to go up by 9
.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).getCount();
// 9 alter commands above.
assertEquals("Unexpected number of self-events generated",
numberOfSelfEventsBefore + 9, selfEventsCountAfter);
}
@@ -2989,12 +2921,7 @@ public class MetastoreEventsProcessorTest {
}
/**
* Create DML request to Catalog
* @param dBName
* @param tableName
* @param redacted_sql_stmt
* @param created_partitions
* @return
* Create DML request to Catalog.
*/
private TUpdateCatalogRequest createTestTUpdateCatalogRequest(String dBName,
String tableName, String redacted_sql_stmt,

View File

@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.metastore.api.NotificationEventResponse;
import org.apache.impala.catalog.CatalogException;
import org.apache.impala.catalog.CatalogServiceCatalog;
import org.apache.impala.catalog.MetaStoreClientPool.MetaStoreClient;
import org.apache.impala.service.CatalogOpExecutor;
import org.apache.thrift.TException;
/**
@@ -32,10 +33,11 @@ import org.apache.thrift.TException;
* functionality of MetastoreEventsProcessor
*/
public class SynchronousHMSEventProcessorForTests extends MetastoreEventsProcessor {
SynchronousHMSEventProcessorForTests(
CatalogServiceCatalog catalog, long startSyncFromId, long pollingFrequencyInSec)
throws CatalogException {
super(catalog, startSyncFromId, pollingFrequencyInSec);
CatalogOpExecutor catalogOpExecutor, long startSyncFromId,
long pollingFrequencyInSec) throws CatalogException {
super(catalogOpExecutor, startSyncFromId, pollingFrequencyInSec);
}
@Override

View File

@@ -0,0 +1,70 @@
// 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.impala.catalog.metastore;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
import org.apache.impala.authorization.NoopAuthorizationFactory;
import org.apache.impala.authorization.NoopAuthorizationFactory.NoopAuthorizationManager;
import org.apache.impala.catalog.CatalogServiceCatalog;
import org.apache.impala.service.CatalogOpExecutor;
import org.apache.impala.testutil.CatalogServiceTestCatalog;
import org.apache.impala.testutil.CatalogTestMetastoreServer;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
/**
* Base class for Catalog metastore server tests which sets up a Catalog metastore
* server endpoint at a random available port.
*/
public abstract class AbstractCatalogMetastoreTest {
protected static CatalogServiceCatalog catalog_;
protected static CatalogOpExecutor catalogOpExecutor_;
protected static CatalogMetastoreServer catalogMetastoreServer_;
protected static HiveMetaStoreClient catalogHmsClient_;
protected static final Configuration CONF = MetastoreConf.newMetastoreConf();
@BeforeClass
public static void setup() throws Exception {
catalog_ = CatalogServiceTestCatalog.create();
catalogOpExecutor_ = new CatalogOpExecutor(catalog_,
new NoopAuthorizationFactory().getAuthorizationConfig(),
new NoopAuthorizationManager());
catalogMetastoreServer_ = new CatalogTestMetastoreServer(
catalogOpExecutor_);
catalog_.setCatalogMetastoreServer(catalogMetastoreServer_);
catalogMetastoreServer_.start();
MetastoreConf.setVar(CONF, ConfVars.THRIFT_URIS,
"thrift://localhost:" + catalogMetastoreServer_.getPort());
// metastore clients which connect to catalogd's HMS endpoint need this
// configuration set since the forwarded HMS call use catalogd's HMS client
// not the end-user's UGI.
CONF.set("hive.metastore.execute.setugi", "false");
catalogHmsClient_ = new HiveMetaStoreClient(CONF);
}
@AfterClass
public static void cleanUp() throws Exception {
catalogMetastoreServer_.stop();
catalog_.close();
}
}

View File

@@ -28,49 +28,19 @@ import java.util.Arrays;
import java.util.Comparator;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
import org.apache.hadoop.hive.metastore.api.GetPartitionsByNamesRequest;
import org.apache.hadoop.hive.metastore.api.GetPartitionsByNamesResult;
import org.apache.hadoop.hive.metastore.api.Partition;
import org.apache.hadoop.hive.metastore.api.Table;
import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
import org.apache.impala.catalog.CatalogServiceCatalog;
import org.apache.impala.catalog.HdfsPartition;
import org.apache.impala.catalog.HdfsPartition.FileBlock;
import org.apache.impala.catalog.HdfsPartition.FileDescriptor;
import org.apache.impala.catalog.HdfsTable;
import org.apache.impala.fb.FbFileBlock;
import org.apache.impala.testutil.CatalogServiceTestCatalog;
import org.apache.impala.testutil.CatalogServiceTestCatalog.CatalogServiceTestHMSCatalog;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
public class CatalogHmsFileMetadataTest {
private static CatalogServiceCatalog catalog_;
private static HiveMetaStoreClient catalogHmsClient_;
private static final Configuration CONF = MetastoreConf.newMetastoreConf();
@BeforeClass
public static void setup() throws Exception {
catalog_ = CatalogServiceTestCatalog.createTestCatalogMetastoreServer();
MetastoreConf.setVar(CONF, ConfVars.THRIFT_URIS,
"thrift://localhost:" + ((CatalogServiceTestHMSCatalog) catalog_).getPort());
// metastore clients which connect to catalogd's HMS endpoint need this
// configuration set since the forwarded HMS call use catalogd's HMS client
// not the end-user's UGI.
CONF.set("hive.metastore.execute.setugi", "false");
catalogHmsClient_ = new HiveMetaStoreClient(CONF);
}
@AfterClass
public static void cleanUp() throws Exception {
catalog_.close();
}
public class CatalogHmsFileMetadataTest extends AbstractCatalogMetastoreTest {
/**
* The test fetches partitions of a table over HMS API and then compares if the
* deserialized filemetadata from the response matches with what we have in catalogd.

View File

@@ -26,46 +26,17 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
import org.apache.hadoop.hive.metastore.api.GetPartitionsByNamesRequest;
import org.apache.hadoop.hive.metastore.api.GetPartitionsByNamesResult;
import org.apache.hadoop.hive.metastore.api.Partition;
import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
import org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars;
import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
import org.apache.impala.catalog.CatalogServiceCatalog;
import org.apache.impala.catalog.HdfsPartition;
import org.apache.impala.catalog.HdfsPartition.FileDescriptor;
import org.apache.impala.catalog.HdfsTable;
import org.apache.impala.service.BackendConfig;
import org.apache.impala.testutil.CatalogServiceTestCatalog;
import org.apache.impala.testutil.CatalogServiceTestCatalog.CatalogServiceTestHMSCatalog;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
public class EnableCatalogdHmsCacheFlagTest {
private static CatalogServiceCatalog catalog_;
private static HiveMetaStoreClient catalogHmsClient_;
private static final Configuration CONF = MetastoreConf.newMetastoreConf();
@BeforeClass
public static void setup() throws Exception {
catalog_ = CatalogServiceTestCatalog.createTestCatalogMetastoreServer();
MetastoreConf.setVar(CONF, ConfVars.THRIFT_URIS,
"thrift://localhost:" + ((CatalogServiceTestHMSCatalog) catalog_).getPort());
// metastore clients which connect to catalogd's HMS endpoint need this
// configuration set since the forwarded HMS call use catalogd's HMS client
// not the end-user's UGI.
CONF.set("hive.metastore.execute.setugi", "false");
catalogHmsClient_ = new HiveMetaStoreClient(CONF);
}
@AfterClass
public static void cleanUp() throws Exception {
catalog_.close();
}
public class EnableCatalogdHmsCacheFlagTest extends AbstractCatalogMetastoreTest {
/**
* The test fetches partitions of a table over HMS API and then compares if the

View File

@@ -17,7 +17,6 @@
package org.apache.impala.testutil;
import java.net.ServerSocket;
import org.apache.impala.authorization.AuthorizationFactory;
import org.apache.impala.authorization.NoopAuthorizationFactory;
import org.apache.impala.authorization.AuthorizationPolicy;
@@ -26,6 +25,8 @@ import org.apache.impala.catalog.CatalogException;
import org.apache.impala.catalog.CatalogServiceCatalog;
import org.apache.impala.catalog.MetaStoreClientPool;
import org.apache.impala.catalog.metastore.CatalogMetastoreServer;
import org.apache.impala.catalog.events.NoOpEventProcessor;
import org.apache.impala.catalog.metastore.NoOpCatalogMetastoreServer;
import org.apache.impala.compat.MetastoreShim;
import org.apache.impala.common.ImpalaException;
import org.apache.impala.service.FeSupport;
@@ -40,7 +41,7 @@ import java.util.UUID;
* for testing.
*/
public class CatalogServiceTestCatalog extends CatalogServiceCatalog {
public CatalogServiceTestCatalog(boolean loadInBackground, int numLoadingThreads,
private CatalogServiceTestCatalog(boolean loadInBackground, int numLoadingThreads,
TUniqueId catalogServiceId, MetaStoreClientPool metaStoreClientPool)
throws ImpalaException {
super(loadInBackground, numLoadingThreads, catalogServiceId,
@@ -57,30 +58,22 @@ public class CatalogServiceTestCatalog extends CatalogServiceCatalog {
return createWithAuth(new NoopAuthorizationFactory());
}
public static CatalogServiceCatalog createWithAuth(AuthorizationFactory authzFactory) {
return createWithAuth(authzFactory, false);
}
/**
* Creates a catalog server that reads authorization policy metadata from the
* authorization config.
*/
public static CatalogServiceCatalog createWithAuth(AuthorizationFactory authzFactory,
boolean startCatalogHms) {
public static CatalogServiceCatalog createWithAuth(AuthorizationFactory authzFactory) {
FeSupport.loadLibrary();
CatalogServiceCatalog cs;
try {
if (MetastoreShim.getMajorVersion() > 2) {
MetastoreShim.setHiveClientCapabilities();
}
if (startCatalogHms) {
cs = new CatalogServiceTestHMSCatalog(false, 16, new TUniqueId(),
new MetaStoreClientPool(0, 0));
} else {
cs = new CatalogServiceTestCatalog(false, 16, new TUniqueId(),
new MetaStoreClientPool(0, 0));
}
cs = new CatalogServiceTestCatalog(false, 16, new TUniqueId(),
new MetaStoreClientPool(0, 0));
cs.setAuthzManager(authzFactory.newAuthorizationManager(cs));
cs.setMetastoreEventProcessor(NoOpEventProcessor.getInstance());
cs.setCatalogMetastoreServer(NoOpCatalogMetastoreServer.INSTANCE);
cs.reset();
} catch (ImpalaException e) {
throw new IllegalStateException(e.getMessage(), e);
@@ -104,80 +97,11 @@ public class CatalogServiceTestCatalog extends CatalogServiceCatalog {
CatalogServiceCatalog cs = new CatalogServiceTestCatalog(false, 16,
new TUniqueId(), new EmbeddedMetastoreClientPool(0, derbyPath));
cs.setAuthzManager(new NoopAuthorizationManager());
cs.setMetastoreEventProcessor(NoOpEventProcessor.getInstance());
cs.reset();
return cs;
}
private static class CatalogTestMetastoreServer extends CatalogMetastoreServer {
private final int port;
public CatalogTestMetastoreServer(
CatalogServiceCatalog catalogServiceCatalog) throws ImpalaException {
super(catalogServiceCatalog);
try {
port = getRandomPort();
} catch (Exception e) {
throw new CatalogException(e.getMessage());
}
}
@Override
public int getPort() {
return port;
}
private static int getRandomPort() throws Exception {
for (int i=0; i<5; i++) {
ServerSocket serverSocket = null;
try {
serverSocket = new ServerSocket(0);
return serverSocket.getLocalPort();
} finally {
if (serverSocket != null) serverSocket.close();
}
}
throw new Exception("Could not find a free port");
}
}
public static class CatalogServiceTestHMSCatalog extends CatalogServiceTestCatalog {
private CatalogTestMetastoreServer metastoreServer;
public CatalogServiceTestHMSCatalog(boolean loadInBackground, int numLoadingThreads,
TUniqueId catalogServiceId,
MetaStoreClientPool metaStoreClientPool) throws ImpalaException {
super(loadInBackground, numLoadingThreads, catalogServiceId, metaStoreClientPool);
}
@Override
protected CatalogMetastoreServer getCatalogMetastoreServer() {
synchronized (this) {
if (metastoreServer != null) return metastoreServer;
try {
metastoreServer = new CatalogTestMetastoreServer(this);
} catch (ImpalaException e) {
return null;
}
return metastoreServer;
}
}
public int getPort() { return metastoreServer.getPort(); }
@Override
public void close() {
super.close();
try {
metastoreServer.stop();
} catch (CatalogException e) {
// ignored
}
}
}
public static CatalogServiceCatalog createTestCatalogMetastoreServer()
throws ImpalaException {
return createWithAuth(new NoopAuthorizationFactory(), true);
}
@Override
public AuthorizationPolicy getAuthPolicy() { return authPolicy_; }
}

View File

@@ -0,0 +1,62 @@
// 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.impala.testutil;
import java.net.ServerSocket;
import org.apache.impala.catalog.CatalogException;
import org.apache.impala.catalog.metastore.CatalogMetastoreServer;
import org.apache.impala.common.ImpalaException;
import org.apache.impala.service.CatalogOpExecutor;
/**
* Test Catalog metastore server which starts on a random port which is available.
*/
public class CatalogTestMetastoreServer extends CatalogMetastoreServer {
private final int port_;
public CatalogTestMetastoreServer(
CatalogOpExecutor catalogOpExecutor) throws ImpalaException {
super(catalogOpExecutor);
try {
port_ = getRandomPort();
} catch (Exception e) {
throw new CatalogException(e.getMessage());
}
}
@Override
public int getPort() {
return port_;
}
private static int getRandomPort() throws Exception {
for (int i = 0; i < 5; i++) {
ServerSocket serverSocket = null;
try {
serverSocket = new ServerSocket(0);
return serverSocket.getLocalPort();
} finally {
if (serverSocket != null) {
serverSocket.close();
}
}
}
throw new Exception("Could not find a free port");
}
}

View File

@@ -14,7 +14,7 @@
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
import logging
import random
import string
import pytest
@@ -93,118 +93,124 @@ class TestEventProcessing(CustomClusterTestSuite):
@CustomClusterTestSuite.with_args(catalogd_args="--hms_event_polling_interval_s=1")
@SkipIfHive2.acid
def test_transactional_insert_events(self):
def test_transactional_insert_events(self, unique_database):
"""Executes 'run_test_insert_events' for transactional tables.
"""
self.run_test_insert_events(is_transactional=True)
self.run_test_insert_events(unique_database, is_transactional=True)
@CustomClusterTestSuite.with_args(catalogd_args="--hms_event_polling_interval_s=1")
def test_insert_events(self):
def test_insert_events(self, unique_database):
"""Executes 'run_test_insert_events' for non-transactional tables.
"""
self.run_test_insert_events()
self.run_test_insert_events(unique_database)
def run_test_insert_events(self, is_transactional=False):
def run_test_insert_events(self, unique_database, is_transactional=False):
"""Test for insert event processing. Events are created in Hive and processed in
Impala. The following cases are tested :
Insert into table --> for partitioned and non-partitioned table
Insert overwrite table --> for partitioned and non-partitioned table
Insert into partition --> for partitioned table
"""
db_name = self.__get_random_name("insert_event_db_")
tblproperties = self.__get_transactional_tblproperties(is_transactional)
with HiveDbWrapper(self, db_name):
# Test table with no partitions.
test_tbl_name = 'tbl_insert_nopart'
self.run_stmt_in_hive("drop table if exists %s.%s" % (db_name, test_tbl_name))
self.run_stmt_in_hive("create table %s.%s (id int, val int) %s"
% (db_name, test_tbl_name, tblproperties))
EventProcessorUtils.wait_for_event_processing(self)
# Test CTAS and insert by Impala with empty results (IMPALA-10765).
self.execute_query("create table {db}.ctas_tbl {prop} as select * from {db}.{tbl}"
.format(db=db_name, tbl=test_tbl_name, prop=tblproperties))
self.execute_query("insert into {db}.ctas_tbl select * from {db}.{tbl}"
.format(db=db_name, tbl=test_tbl_name))
# Test insert into table, this will fire an insert event.
self.run_stmt_in_hive("insert into %s.%s values(101, 200)"
% (db_name, test_tbl_name))
# With MetastoreEventProcessor running, the insert event will be processed. Query
# the table from Impala.
EventProcessorUtils.wait_for_event_processing(self)
# Verify that the data is present in Impala.
data = self.execute_scalar("select * from %s.%s" % (db_name, test_tbl_name))
assert data.split('\t') == ['101', '200']
# Test table with no partitions.
tbl_insert_nopart = 'tbl_insert_nopart'
self.run_stmt_in_hive(
"drop table if exists %s.%s" % (unique_database, tbl_insert_nopart))
tblproperties = ""
if is_transactional:
tblproperties = "tblproperties ('transactional'='true'," \
"'transactional_properties'='insert_only')"
self.run_stmt_in_hive("create table %s.%s (id int, val int) %s"
% (unique_database, tbl_insert_nopart, tblproperties))
EventProcessorUtils.wait_for_event_processing(self)
# Test CTAS and insert by Impala with empty results (IMPALA-10765).
self.execute_query("create table {db}.ctas_tbl {prop} as select * from {db}.{tbl}"
.format(db=unique_database, tbl=tbl_insert_nopart, prop=tblproperties))
self.execute_query("insert into {db}.ctas_tbl select * from {db}.{tbl}"
.format(db=unique_database, tbl=tbl_insert_nopart))
# Test insert into table, this will fire an insert event.
self.run_stmt_in_hive("insert into %s.%s values(101, 200)"
% (unique_database, tbl_insert_nopart))
# With MetastoreEventProcessor running, the insert event will be processed. Query the
# table from Impala.
EventProcessorUtils.wait_for_event_processing(self)
# Verify that the data is present in Impala.
data = self.execute_scalar("select * from %s.%s" %
(unique_database, tbl_insert_nopart))
assert data.split('\t') == ['101', '200']
# Test insert overwrite. Overwrite the existing value.
self.run_stmt_in_hive("insert overwrite table %s.%s values(101, 201)"
% (db_name, test_tbl_name))
# Make sure the event has been processed.
EventProcessorUtils.wait_for_event_processing(self)
# Verify that the data is present in Impala.
data = self.execute_scalar("select * from %s.%s" % (db_name, test_tbl_name))
assert data.split('\t') == ['101', '201']
# Test insert overwrite by Impala with empty results (IMPALA-10765).
self.execute_query("insert overwrite {db}.{tbl} select * from {db}.ctas_tbl"
.format(db=db_name, tbl=test_tbl_name))
result = self.execute_query("select * from {db}.{tbl}"
.format(db=db_name, tbl=test_tbl_name))
assert len(result.data) == 0
# Test insert overwrite. Overwrite the existing value.
self.run_stmt_in_hive("insert overwrite table %s.%s values(101, 201)"
% (unique_database, tbl_insert_nopart))
# Make sure the event has been processed.
EventProcessorUtils.wait_for_event_processing(self)
# Verify that the data is present in Impala.
data = self.execute_scalar("select * from %s.%s" %
(unique_database, tbl_insert_nopart))
assert data.split('\t') == ['101', '201']
# Test insert overwrite by Impala with empty results (IMPALA-10765).
self.execute_query("insert overwrite {db}.{tbl} select * from {db}.ctas_tbl"
.format(db=unique_database, tbl=tbl_insert_nopart))
result = self.execute_query("select * from {db}.{tbl}"
.format(db=unique_database, tbl=tbl_insert_nopart))
assert len(result.data) == 0
# Test partitioned table.
test_part_tblname = 'tbl_insert_part'
self.run_stmt_in_hive("drop table if exists %s.%s" % (db_name, test_part_tblname))
self.run_stmt_in_hive("create table %s.%s (id int, name string) "
"partitioned by(day int, month int, year int) %s"
% (db_name, test_part_tblname, tblproperties))
EventProcessorUtils.wait_for_event_processing(self)
# Test insert overwrite by Impala with empty results (IMPALA-10765).
self.execute_query("create table {db}.ctas_part partitioned by (day, month, year) "
"{prop} as select * from {db}.{tbl}"
.format(db=db_name, tbl=test_part_tblname, prop=tblproperties))
self.execute_query("insert into {db}.ctas_part partition(day=0, month=0, year=0) "
"select id, name from {db}.{tbl}"
.format(db=db_name, tbl=test_part_tblname))
# Insert data into partitions.
self.run_stmt_in_hive("insert into %s.%s partition(day=28, month=03, year=2019)"
"values(101, 'x')" % (db_name, test_part_tblname))
# Make sure the event has been processed.
EventProcessorUtils.wait_for_event_processing(self)
# Verify that the data is present in Impala.
data = self.execute_scalar("select * from %s.%s" % (db_name, test_part_tblname))
assert data.split('\t') == ['101', 'x', '28', '3', '2019']
# Test partitioned table.
tbl_insert_part = 'tbl_insert_part'
self.run_stmt_in_hive("drop table if exists %s.%s"
% (unique_database, tbl_insert_part))
self.run_stmt_in_hive("create table %s.%s (id int, name string) "
"partitioned by(day int, month int, year int) %s"
% (unique_database, tbl_insert_part, tblproperties))
EventProcessorUtils.wait_for_event_processing(self)
# Test insert overwrite by Impala with empty results (IMPALA-10765).
self.execute_query(
"create table {db}.ctas_part partitioned by (day, month, year) {prop} as "
"select * from {db}.{tbl}".format(db=unique_database, tbl=tbl_insert_part,
prop=tblproperties))
self.execute_query(
"insert into {db}.ctas_part partition(day=0, month=0, year=0) select id, "
"name from {db}.{tbl}".format(db=unique_database, tbl=tbl_insert_part))
# Insert data into partitions.
self.run_stmt_in_hive("insert into %s.%s partition(day=28, month=03, year=2019)"
"values(101, 'x')" % (unique_database, tbl_insert_part))
# Make sure the event has been processed.
EventProcessorUtils.wait_for_event_processing(self)
# Verify that the data is present in Impala.
data = self.execute_scalar("select * from %s.%s" % (unique_database, tbl_insert_part))
assert data.split('\t') == ['101', 'x', '28', '3', '2019']
# Test inserting into existing partitions.
self.run_stmt_in_hive("insert into %s.%s partition(day=28, month=03, year=2019)"
"values(102, 'y')" % (db_name, test_part_tblname))
EventProcessorUtils.wait_for_event_processing(self)
# Verify that the data is present in Impala.
data = self.execute_scalar("select count(*) from %s.%s where day=28 and month=3 "
"and year=2019" % (db_name, test_part_tblname))
assert data.split('\t') == ['2']
# Test inserting into existing partitions by Impala with empty results
# (IMPALA-10765).
self.execute_query("insert into {db}.{tbl} partition(day=28, month=03, year=2019) "
"select id, name from {db}.ctas_part"
.format(db=db_name, tbl=test_part_tblname))
# Test inserting into existing partitions.
self.run_stmt_in_hive("insert into %s.%s partition(day=28, month=03, year=2019)"
"values(102, 'y')" % (unique_database, tbl_insert_part))
EventProcessorUtils.wait_for_event_processing(self)
# Verify that the data is present in Impala.
data = self.execute_scalar("select count(*) from %s.%s where day=28 and month=3 "
"and year=2019" % (unique_database, tbl_insert_part))
assert data.split('\t') == ['2']
# Test inserting into existing partitions by Impala with empty results
# (IMPALA-10765).
self.execute_query("insert into {db}.{tbl} partition(day=28, month=03, year=2019) "
"select id, name from {db}.ctas_part"
.format(db=unique_database, tbl=tbl_insert_part))
# Test insert overwrite into existing partitions
self.run_stmt_in_hive("insert overwrite table %s.%s partition(day=28, month=03, "
"year=2019)" "values(101, 'z')" % (db_name, test_part_tblname))
EventProcessorUtils.wait_for_event_processing(self)
# Verify that the data is present in Impala.
data = self.execute_scalar("select * from %s.%s where day=28 and month=3 and"
" year=2019 and id=101" % (db_name, test_part_tblname))
assert data.split('\t') == ['101', 'z', '28', '3', '2019']
# Test insert overwrite into existing partitions by Impala with empty results
# (IMPALA-10765).
self.execute_query("insert overwrite {db}.{tbl} "
"partition(day=28, month=03, year=2019) "
"select id, name from {db}.ctas_part"
.format(db=db_name, tbl=test_part_tblname))
result = self.execute_query("select * from {db}.{tbl} "
"where day=28 and month=3 and year=2019"
.format(db=db_name, tbl=test_part_tblname))
assert len(result.data) == 0
# Test insert overwrite into existing partitions
self.run_stmt_in_hive("insert overwrite table %s.%s partition(day=28, month=03, "
"year=2019)" "values(101, 'z')" % (unique_database, tbl_insert_part))
EventProcessorUtils.wait_for_event_processing(self)
# Verify that the data is present in Impala.
data = self.execute_scalar("select * from %s.%s where day=28 and month=3 and"
" year=2019 and id=101" % (unique_database, tbl_insert_part))
assert data.split('\t') == ['101', 'z', '28', '3', '2019']
# Test insert overwrite into existing partitions by Impala with empty results
# (IMPALA-10765).
self.execute_query("insert overwrite {db}.{tbl} "
"partition(day=28, month=03, year=2019) "
"select id, name from {db}.ctas_part"
.format(db=unique_database, tbl=tbl_insert_part))
result = self.execute_query("select * from {db}.{tbl} "
"where day=28 and month=3 and year=2019"
.format(db=unique_database, tbl=tbl_insert_part))
assert len(result.data) == 0
@CustomClusterTestSuite.with_args(catalogd_args="--hms_event_polling_interval_s=1")
def test_iceberg_inserts(self):
@@ -244,6 +250,112 @@ class TestEventProcessing(CustomClusterTestSuite):
self.__run_self_events_test(unique_database, True)
self.__run_self_events_test(unique_database, False)
@pytest.mark.execute_serially
@CustomClusterTestSuite.with_args(
catalogd_args="--hms_event_polling_interval_s=1",
cluster_size=1)
def test_create_drop_events(self, unique_database):
self.__run_create_drop_test(unique_database, "database")
self.__run_create_drop_test(unique_database, "table")
self.__run_create_drop_test(unique_database, "table", True)
self.__run_create_drop_test(unique_database, "table", True, True)
self.__run_create_drop_test(unique_database, "partition")
@pytest.mark.execute_serially
@CustomClusterTestSuite.with_args(
impalad_args="--use_local_catalog=true",
catalogd_args="--catalog_topic_mode=minimal --hms_event_polling_interval_s=1",
cluster_size=1)
def test_local_catalog_create_drop_events(self, unique_database):
self.__run_create_drop_test(unique_database, "database")
self.__run_create_drop_test(unique_database, "table")
self.__run_create_drop_test(unique_database, "table", True)
self.__run_create_drop_test(unique_database, "table", True, True)
self.__run_create_drop_test(unique_database, "partition")
def __run_create_drop_test(self, db, type, rename=False, rename_db=False):
if type == "table":
if not rename:
queries = [
"create table {0}.test_{1} (i int)".format(db, 1),
"drop table {0}.test_{1}".format(db, 1)
]
else:
db_1 = "{}_1".format(db)
if rename_db:
self.execute_query_expect_success(self.create_impala_client(),
"drop database if exists {0} cascade".format(db_1))
self.execute_query_expect_success(self.create_impala_client(),
"create database {0}".format(db_1))
self.execute_query_expect_success(self.create_impala_client(),
"create table if not exists {0}.rename_test_1 (i int)".format(db))
if rename_db:
queries = [
"alter table {0}.rename_test_1 rename to {1}.rename_test_1".format(db, db_1),
"alter table {0}.rename_test_1 rename to {1}.rename_test_1".format(db_1, db)
]
else:
queries = [
"alter table {0}.rename_test_1 rename to {0}.rename_test_2".format(db),
"alter table {0}.rename_test_2 rename to {0}.rename_test_1".format(db)
]
create_metric_name = "tables-added"
removed_metric_name = "tables-removed"
elif type == "database":
self.execute_query_expect_success(self.create_impala_client(),
"drop database if exists {0}".format("test_create_drop_db"))
queries = [
"create database {db}".format(db="test_create_drop_db"),
"drop database {db}".format(db="test_create_drop_db")
]
create_metric_name = "databases-added"
removed_metric_name = "databases-removed"
else:
tbl_name = "test_create_drop_partition"
self.execute_query_expect_success(self.create_impala_client(),
"create table {db}.{tbl} (c int) partitioned by (p int)".format(
db=db, tbl=tbl_name))
queries = [
"alter table {db}.{tbl} add partition (p=1)".format(db=db, tbl=tbl_name),
"alter table {db}.{tbl} drop partition (p=1)".format(db=db, tbl=tbl_name)
]
create_metric_name = "partitions-added"
removed_metric_name = "partitions-removed"
# get the metric before values
EventProcessorUtils.wait_for_event_processing(self)
create_metric_val_before = EventProcessorUtils.\
get_event_processor_metric(create_metric_name, 0)
removed_metric_val_before = EventProcessorUtils.\
get_event_processor_metric(removed_metric_name, 0)
events_skipped_before = EventProcessorUtils.\
get_event_processor_metric('events-skipped', 0)
num_iters = 200
for iter in xrange(num_iters):
for q in queries:
try:
self.execute_query_expect_success(self.create_impala_client(), q)
except Exception as e:
print("Failed in {} iterations. Error {}".format(iter, str(e)))
raise
EventProcessorUtils.wait_for_event_processing(self)
create_metric_val_after = EventProcessorUtils. \
get_event_processor_metric(create_metric_name, 0)
removed_metric_val_after = EventProcessorUtils. \
get_event_processor_metric(removed_metric_name, 0)
events_skipped_after = EventProcessorUtils. \
get_event_processor_metric('events-skipped', 0)
num_delete_event_entries = EventProcessorUtils. \
get_event_processor_metric('delete-event-log-size', 0)
assert EventProcessorUtils.get_event_processor_status() == "ACTIVE"
# None of the queries above should actually trigger a add/remove object from events
assert int(create_metric_val_after) == int(create_metric_val_before)
assert int(removed_metric_val_after) == int(removed_metric_val_before)
# each query set generates 2 events and both of them should be skipped
assert int(events_skipped_after) == num_iters * 2 + int(events_skipped_before)
# make sure that there are no more entries in the delete event log
assert int(num_delete_event_entries) == 0
@CustomClusterTestSuite.with_args(catalogd_args="--hms_event_polling_interval_s=1")
def test_event_based_replication(self):
self.__run_event_based_replication_tests()
@@ -499,13 +611,16 @@ class TestEventProcessing(CustomClusterTestSuite):
self.client.execute(
"create external table {0}.{1} like functional.alltypes location '{2}'".format(
db_name, recover_tbl_name, alltypes_tab_location))
self.client.execute("refresh {0}.{1}".format(db_name, recover_tbl_name))
if use_impala:
queries = self.__get_impala_test_queries(db_name, recover_tbl_name)
# some queries do not trigger self-event evaluation (creates and drops) however,
# its still good to confirm that we don't do unnecessary refreshes in such cases
# For such queries we use a different metrics events-skipped to confirm that these
# events are skipped.
for stmt in queries[False]:
self.__exec_sql_and_check_selfevent_counter(stmt, use_impala, False)
# All the queries with True key should confirm that the self-events-skipped counter
# All the queries with True key should confirm that the events-skipped counter
# is also incremented
for stmt in queries[True]:
self.__exec_sql_and_check_selfevent_counter(stmt, use_impala)
@@ -520,10 +635,11 @@ class TestEventProcessing(CustomClusterTestSuite):
acid_no_part_tbl_name = self.__get_random_name("acid_no_part_tbl_")
tbl2 = self.__get_random_name("tbl_")
view_name = self.__get_random_name("view_")
view2 = self.__get_random_name("view_")
# create a empty table for both partitioned and unpartitioned case for testing insert
# events
empty_unpartitioned_tbl = self.__get_random_name("insert_test_tbl_")
empty_partitioned_tbl = self.__get_random_name("insert_test_parttbl_")
empty_unpartitioned_tbl = self.__get_random_name("empty_unpart_tbl_")
empty_partitioned_tbl = self.__get_random_name("empty_parttbl_")
self.client.execute(
"create table {0}.{1} (c1 int)".format(db_name, empty_unpartitioned_tbl))
self.client.execute(
@@ -531,7 +647,7 @@ class TestEventProcessing(CustomClusterTestSuite):
empty_partitioned_tbl))
acid_props = self.__get_transactional_tblproperties(True)
self_event_test_queries = {
# Queries which will increment the self-events-skipped counter
# Queries which will increment the events-skipped counter
True: [
# ALTER_DATABASE case
"comment on database {0} is 'self-event test database'".format(db_name),
@@ -550,36 +666,39 @@ class TestEventProcessing(CustomClusterTestSuite):
"alter table {0}.{1} DROP COLUMN c3".format(db_name, tbl_name),
"alter table {0}.{1} set owner user `test-user`".format(db_name, tbl_name),
"alter table {0}.{1} set owner role `test-role`".format(db_name, tbl_name),
"alter table {0}.{1} rename to {0}.{2}".format(db_name, tbl_name, tbl2),
"alter view {0}.{1} set owner user `test-view-user`".format(db_name, view_name),
"alter view {0}.{1} set owner role `test-view-role`".format(db_name, view_name),
"alter view {0}.{1} rename to {0}.{2}".format(db_name, view_name,
self.__get_random_name("view_")),
# ADD_PARTITION cases
# dynamic partition insert (creates new partitions)
"insert into table {0}.{1} partition (year,month) "
"select * from functional.alltypessmall".format(db_name, tbl2),
# add partition
"alter table {0}.{1} add if not exists partition (year=1111, month=1)".format(
db_name, tbl2),
# compute stats will generates ALTER_PARTITION
"compute stats {0}.{1}".format(db_name, tbl2),
"alter table {0}.{1} recover partitions".format(db_name, recover_tbl_name),
"compute stats {0}.{1}".format(db_name, tbl_name),
# insert into a existing partition; generates INSERT self-event
"insert into table {0}.{1} partition "
"(year, month) select * from functional.alltypessmall where year=2009 "
"and month=1".format(db_name, tbl2),
"and month=1".format(db_name, tbl_name),
# insert overwrite query from Impala also generates a INSERT self-event
"insert overwrite table {0}.{1} partition "
"(year, month) select * from functional.alltypessmall where year=2009 "
"and month=1".format(db_name, tbl2),
],
# Queries which will not increment the self-events-skipped counter
"and month=1".format(db_name, tbl_name)],
# Queries which will not increment the events-skipped counter
False: [
"create table {0}.{1} like functional.alltypessmall "
"stored as parquet".format(db_name, tbl_name),
"create view {0}.{1} as select * from functional.alltypessmall "
"where year=2009".format(db_name, view_name),
"where year=2009".format(db_name, view_name),
# in case of rename we process it as drop+create and hence
# the events-skipped counter is not updated. Instead if this event is processed,
# it will increment the tables-added and tables-removed counters.
"alter table {0}.{1} rename to {0}.{2}".format(db_name, tbl_name, tbl2),
"alter table {0}.{1} rename to {0}.{2}".format(db_name, tbl2, tbl_name),
"alter view {0}.{1} rename to {0}.{2}".format(db_name, view_name, view2),
"alter view {0}.{1} rename to {0}.{2}".format(db_name, view2, view_name),
# ADD_PARTITION cases
# dynamic partition insert (creates new partitions)
"insert into table {0}.{1} partition (year,month) "
"select * from functional.alltypessmall where month % 2 = 0".format(db_name,
tbl_name),
"insert overwrite table {0}.{1} partition (year,month) "
"select * from functional.alltypessmall where month % 2 = 1".format(db_name,
tbl_name),
# we add this statement below just to make sure that the subsequent statement is
# a no-op
"alter table {0}.{1} add if not exists partition (year=2100, month=1)".format(
@@ -621,19 +740,20 @@ class TestEventProcessing(CustomClusterTestSuite):
# leads to a non-self event that reloads the table
"insert overwrite table {0}.{1} partition (part) select id, int_col "
"from functional.alltypestiny where id=-1".format(db_name, acid_tbl_name),
# the table is empty so the following inserts add 0 rows
"insert overwrite table {0}.{1} partition (part) select id, int_col "
"from functional.alltypestiny".format(db_name, acid_tbl_name),
"insert overwrite {0}.{1} partition(part) select * from {0}.{1}".format(
db_name, acid_tbl_name),
# recover partitions will generate add_partition events
"alter table {0}.{1} recover partitions".format(db_name, recover_tbl_name)
]
}
return self_event_test_queries
def __get_hive_test_queries(self, db_name, recover_tbl_name):
tbl_name = self.__get_random_name("tbl_")
tbl2 = self.__get_random_name("tbl_")
view_name = self.__get_random_name("view_")
tbl_name = self.__get_random_name("hive_test_tbl_")
tbl2 = self.__get_random_name("hive_renamed_tbl_")
view_name = self.__get_random_name("hive_view_")
# we use a custom table schema to make it easier to change columns later in the
# test_queries
self.client.execute("create table {0}.{1} (key int) partitioned by "
@@ -654,79 +774,96 @@ class TestEventProcessing(CustomClusterTestSuite):
"alter table {0}.{1} add columns (value string)".format(db_name, tbl_name),
"alter table {0}.{1} set owner user `test-user`".format(db_name, tbl_name),
"alter table {0}.{1} set owner role `test-role`".format(db_name, tbl_name),
"alter table {0}.{1} rename to {0}.{2}".format(db_name, tbl_name, tbl2),
"alter view {0}.{1} rename to {0}.{2}".format(db_name, view_name,
self.__get_random_name("view_")),
# need to set this config to make sure the dynamic partition insert works below
"set hive.exec.dynamic.partition.mode=nonstrict",
# ADD_PARTITION cases
"insert into table {0}.{1} partition (part=2009) "
"select id as key, string_col as value from functional.alltypessmall".format(
db_name, tbl2),
db_name, tbl_name),
# add partition
"alter table {0}.{1} add if not exists partition (part=1111)".format(
db_name, tbl2),
db_name, tbl_name),
# add existing partition; essentially this is a no-op
"alter table {0}.{1} add if not exists partition (part=1111)".format(
db_name, tbl2),
db_name, tbl_name),
# DROP_PARTITION cases
"alter table {0}.{1} drop if exists partition (part=1111)".format(
db_name, tbl2),
db_name, tbl_name),
# drop non-existing partition; essentially this is a no-op
"alter table {0}.{1} drop if exists partition (part=1111)".format(
db_name, tbl2),
db_name, tbl_name),
# compute stats will generates ALTER_PARTITION
"analyze table {0}.{1} compute statistics for columns".format(db_name, tbl2),
"msck repair table {0}.{1}".format(db_name, recover_tbl_name)
"analyze table {0}.{1} compute statistics for columns".format(db_name, tbl_name),
"msck repair table {0}.{1}".format(db_name, recover_tbl_name),
# we rename in the end since impala will have the new table in unloaded
# state after rename and hence any events later will be ignored anyways.
"alter table {0}.{1} rename to {0}.{2}".format(db_name, tbl_name, tbl2),
"alter view {0}.{1} rename to {0}.{2}".format(db_name, view_name,
self.__get_random_name("view_")),
]
return self_event_test_queries
@staticmethod
def __get_self_event_metrics():
"""
Gets the self-events-skipped, tables-refreshed and partitions-refreshed metric values
Gets the tables-refreshed, partitions-refreshed and events-skipped metric values
from Metastore EventsProcessor
"""
tbls_refreshed_count = EventProcessorUtils.get_event_processor_metric(
'tables-refreshed', 0)
partitions_refreshed_count = EventProcessorUtils.get_event_processor_metric(
'partitions-refreshed', 0)
self_events_count = EventProcessorUtils.get_event_processor_metric(
'self-events-skipped', 0)
return int(self_events_count), int(tbls_refreshed_count), int(
partitions_refreshed_count)
events_skipped_count = EventProcessorUtils.get_event_processor_metric(
'events-skipped', 0)
return int(tbls_refreshed_count), int(partitions_refreshed_count), \
int(events_skipped_count)
def __exec_sql_and_check_selfevent_counter(self, stmt, use_impala_client,
check_self_event_counter=True):
check_events_skipped_counter=True):
"""
Method runs a given query statement using a impala client or hive client based on the
argument use_impala_client and confirms if the self-event related counters are as
expected based on whether we expect a self-event or not. If the
check_self_event_counter is False it skips checking the self-events-skipped metric.
check_self_event_counter is False it skips checking the events-skipped metric.
"""
self_events, tbls_refreshed, partitions_refreshed = self.__get_self_event_metrics()
EventProcessorUtils.wait_for_event_processing(self)
tbls_refreshed, partitions_refreshed,\
events_skipped = self.__get_self_event_metrics()
last_synced_event = EventProcessorUtils.get_last_synced_event_id()
logging.info("Running statement in {1}: {0}".format(stmt,
"impala" if use_impala_client else "hive"))
if not use_impala_client:
self.run_stmt_in_hive(stmt)
else:
self.client.execute(stmt)
EventProcessorUtils.wait_for_event_processing(self)
self_events_after, tbls_refreshed_after, partitions_refreshed_after = \
self.__get_self_event_metrics()
tbls_refreshed_after, partitions_refreshed_after,\
events_skipped_after = self.__get_self_event_metrics()
last_synced_event_after = EventProcessorUtils.get_last_synced_event_id()
# we assume that any event which comes due to stmts run from impala-client are
# self-events
logging.info(
"Event id before {0} event id after {1}".format(last_synced_event,
last_synced_event_after))
if use_impala_client:
# self-event counter must increase if this is a self-event if
# check_self_event_counter is set
if check_self_event_counter:
assert self_events_after > self_events
# some of the test queries generate no events at all. If that is the case
# skip the below comparison
if last_synced_event_after > last_synced_event:
if check_events_skipped_counter:
assert events_skipped_after > events_skipped, \
"Failing query(impala={}): {}".format(use_impala_client, stmt)
# if this is a self-event, no table or partitions should be refreshed
assert tbls_refreshed == tbls_refreshed_after
assert partitions_refreshed == partitions_refreshed_after
assert tbls_refreshed == tbls_refreshed_after, \
"Failing query(impala={}): {}".format(use_impala_client, stmt)
assert partitions_refreshed == partitions_refreshed_after, \
"Failing query(impala={}): {}".format(use_impala_client, stmt)
else:
# hive was used to run the stmts, any events generated should not have been deemed
# as self events
assert self_events == self_events_after
assert events_skipped == events_skipped_after
@staticmethod
def __get_random_name(prefix=''):