mirror of
https://github.com/apache/impala.git
synced 2025-12-19 18:12:08 -05:00
IMPALA-13801: Support greatest synced event with hierarchical metastore event processing
It is a follow-up jira/commit to IMPALA-12709. IMPALA-12152 and
IMPALA-12785 are affected when hierarchical metastore event
processing feature is enabled.
Following changes are incorporated with this patch:
1. Added creationTime_ and dispatchTime_ fields in MetastoreEvent
class to store the current time in millisec. They are used to
calculate:
a) Event dispatch time(time between a MetastoreEvent object
creation and when event is moved to inProgressLog_ of
EventExecutorService after dispatching it to a
DbEventExecutor).
b) Event schedule delays incurred at DbEventExecutors and
TableEventExecutors(time between an event moved to
EventExecutorService's inProgressLog_ and before start of
processing event at appropriate DbEventExecutor and
TableEventExecutor).
c) Event process time from EventExecutorService point of
view(time spent in inProgressLog_ before it is moved to
processedLog_).
Logs are added to show the event dispatch time, schedule
delays, process time from EventExecutorService point of
view for each event. Also a log is added to show the time
taken for event's processIfEnabled().
2. Added isDelimiter_ field in MetastoreEvent class to indicate
whether it is a delimiter event. It is set only when
hierarchical event processing is enabled. Delimiter is a kind
of metastore event that do not require event processing.
Delimeter event can be:
a) A CommitTxnEvent that do not have any write event info for
a given transaction.
b) An AbortTxnEvent that do not have write ids for a given
transaction.
c) An IgnoredEvent.
An event is determined and marked as delimiter in
EventExecutorService#dispatch(). They are not queued to a
DbEventExecutor for processing. They are just maintained in
the inProgressLog_ to preserve continuity and correctness in
synchronization tracking. The delimiter events are removed from
inProgressLog_ when their preceding non-delimiter metastore
event is removed from inProgressLog_.
3. Greatest synced event id is computed based on the dispatched
events(inProgressLog_) and processed events(processedLog_) tree
maps. Greatest synced event is the latest event such that all
events with id less than or equal to the latest event are
definitely synced.
4. Lag is calculated as difference between latest event time on HMS
and the greatest synced event time. It is shown in the log.
5. Greatest synced event id is used in IMPALA-12152 changes. When
greatest synced event id becomes greater than or equal to
waitForEventId, all the required events are definitely synced.
6. Event processor is paused gracefully when paused with command in
IMPALA-12785. This ensures that all the fetched events from HMS in
current batch are processed before the event processor is fully
paused. It is necessary to process the current batch of events
because, certain events like AllocWriteIdEvent, AbortTxnEvent and
CommitTxnEvent update table write ids in catalog upon metastore
event object creation. And the table write ids are later updated
to appropriate table object during their event process. Can lead
to inconsistent state of write ids on table objects when paused
abruptly in the middle of current batch of event processing.
7. Added greatest synced event id and event time in events processor
metrics. And updated description of lag, pending events, last
synced event id and event time metrics.
8. Atomically update the event queue and increment outstanding event
count in enqueue methods of both DbProcessor and TableProcessor
so that respective process methods do not process the event until
event is added to queue and outstanding event count is incremented.
Otherwise, event can get processed, outstanding event count gets
decremented before it is incremented in enqueue method.
9. Refactored DbEventExecutor, DbProcessor, TableEventExecutor and
TableProcessor classes to propapage the exception occurred along
with event during event processing. EventProcessException is a
wrapper added to hold reference to event being processed and
exception occurred.
10.Added AcidTableWriteInfo helper class to store table, writeids
and partitions for the transaction id received in CommitTxnEvent.
Testing:
- Added new tests and executed existing end to end tests.
- Have executed the existing tests with hierarchical event processing
enabled.
Change-Id: I26240f36aaf85125428dc39a66a2a1e4d3197e85
Reviewed-on: http://gerrit.cloudera.org:8080/22997
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Reviewed-by: Quanlong Huang <huangquanlong@gmail.com>
This commit is contained in:
committed by
Quanlong Huang
parent
48b38810e8
commit
ebbc67cf40
@@ -66,6 +66,10 @@ string MetastoreEventMetrics::LAST_SYNCED_EVENT_ID_METRIC_NAME =
|
||||
"events-processor.last-synced-event-id";
|
||||
string MetastoreEventMetrics::LAST_SYNCED_EVENT_TIME_METRIC_NAME =
|
||||
"events-processor.last-synced-event-time";
|
||||
string MetastoreEventMetrics::GREATEST_SYNCED_EVENT_ID_METRIC_NAME =
|
||||
"events-processor.greatest-synced-event-id";
|
||||
string MetastoreEventMetrics::GREATEST_SYNCED_EVENT_TIME_METRIC_NAME =
|
||||
"events-processor.greatest-synced-event-time";
|
||||
string MetastoreEventMetrics::LATEST_EVENT_ID_METRIC_NAME =
|
||||
"events-processor.latest-event-id";
|
||||
string MetastoreEventMetrics::LATEST_EVENT_TIME_METRIC_NAME =
|
||||
@@ -98,6 +102,8 @@ DoubleGauge* MetastoreEventMetrics::EVENTS_RECEIVED_5MIN_RATE = nullptr;
|
||||
DoubleGauge* MetastoreEventMetrics::EVENTS_RECEIVED_15MIN_RATE = nullptr;
|
||||
IntCounter* MetastoreEventMetrics::LAST_SYNCED_EVENT_ID = nullptr;
|
||||
IntCounter* MetastoreEventMetrics::LAST_SYNCED_EVENT_TIME = nullptr;
|
||||
IntCounter* MetastoreEventMetrics::GREATEST_SYNCED_EVENT_ID = nullptr;
|
||||
IntCounter* MetastoreEventMetrics::GREATEST_SYNCED_EVENT_TIME = nullptr;
|
||||
IntCounter* MetastoreEventMetrics::LATEST_EVENT_ID = nullptr;
|
||||
IntCounter* MetastoreEventMetrics::LATEST_EVENT_TIME = nullptr;
|
||||
IntCounter* MetastoreEventMetrics::PENDING_EVENTS = nullptr;
|
||||
@@ -150,6 +156,10 @@ void MetastoreEventMetrics::InitMetastoreEventMetrics(MetricGroup* metric_group)
|
||||
event_metrics->AddCounter(LAST_SYNCED_EVENT_ID_METRIC_NAME, 0);
|
||||
LAST_SYNCED_EVENT_TIME =
|
||||
event_metrics->AddCounter(LAST_SYNCED_EVENT_TIME_METRIC_NAME, 0);
|
||||
GREATEST_SYNCED_EVENT_ID =
|
||||
event_metrics->AddCounter(GREATEST_SYNCED_EVENT_ID_METRIC_NAME, 0);
|
||||
GREATEST_SYNCED_EVENT_TIME =
|
||||
event_metrics->AddCounter(GREATEST_SYNCED_EVENT_TIME_METRIC_NAME, 0);
|
||||
LATEST_EVENT_ID =
|
||||
event_metrics->AddCounter(LATEST_EVENT_ID_METRIC_NAME, 0);
|
||||
LATEST_EVENT_TIME =
|
||||
@@ -219,33 +229,38 @@ void MetastoreEventMetrics::refresh(TEventProcessorMetrics* response) {
|
||||
if (response->__isset.last_synced_event_time) {
|
||||
LAST_SYNCED_EVENT_TIME->SetValue(response->last_synced_event_time);
|
||||
}
|
||||
if (response->__isset.greatest_synced_event_id) {
|
||||
GREATEST_SYNCED_EVENT_ID->SetValue(response->greatest_synced_event_id);
|
||||
}
|
||||
if (response->__isset.greatest_synced_event_time) {
|
||||
GREATEST_SYNCED_EVENT_TIME->SetValue(response->greatest_synced_event_time);
|
||||
}
|
||||
if (response->__isset.latest_event_id) {
|
||||
LATEST_EVENT_ID->SetValue(response->latest_event_id);
|
||||
}
|
||||
if (response->__isset.latest_event_time) {
|
||||
LATEST_EVENT_TIME->SetValue(response->latest_event_time);
|
||||
}
|
||||
// last_synced_event_time and greatest_synced_event_time are same when
|
||||
// hierarchical event processing is not enabled. So, using
|
||||
// greatest_synced_event_time instead of last_synced_event_time so that same
|
||||
// code works with and without hierarchical event processing enabled.
|
||||
// last_synced_event_time is 0 at the startup until we have synced any events.
|
||||
if (response->__isset.latest_event_time && response->__isset.last_synced_event_time
|
||||
&& response->last_synced_event_time > 0) {
|
||||
if (response->__isset.latest_event_time
|
||||
&& response->__isset.greatest_synced_event_time
|
||||
&& response->greatest_synced_event_time > 0) {
|
||||
// latest_event_time and last_synced_event_time are updated by different threads.
|
||||
// It's possible that latest_event_time is stale and smaller than
|
||||
// last_synced_event_time. Set the lag to 0 in this case.
|
||||
if (response->latest_event_time <= response->last_synced_event_time) {
|
||||
if (response->latest_event_time <= response->greatest_synced_event_time) {
|
||||
LAG_TIME->SetValue(0);
|
||||
} else {
|
||||
LAG_TIME->SetValue(response->latest_event_time - response->last_synced_event_time);
|
||||
LAG_TIME->SetValue(
|
||||
response->latest_event_time - response->greatest_synced_event_time);
|
||||
}
|
||||
}
|
||||
if (response->__isset.latest_event_id && response->__isset.last_synced_event_id) {
|
||||
// Same as above, latest_event_id and last_synced_event_id are updated by different
|
||||
// threads. Set the value to 0 if latest_event_id is stale.
|
||||
if (response->latest_event_id <= response->last_synced_event_id) {
|
||||
PENDING_EVENTS->SetValue(0);
|
||||
} else {
|
||||
PENDING_EVENTS->SetValue(
|
||||
response->latest_event_id - response->last_synced_event_id);
|
||||
}
|
||||
if (response->__isset.pending_event_count) {
|
||||
PENDING_EVENTS->SetValue(response->pending_event_count);
|
||||
}
|
||||
if (response->__isset.outstanding_event_count) {
|
||||
OUTSTANDING_EVENT_COUNT->SetValue(response->outstanding_event_count);
|
||||
|
||||
@@ -79,6 +79,12 @@ class MetastoreEventMetrics {
|
||||
/// Last metastore event time that the catalog server synced to.
|
||||
static IntCounter* LAST_SYNCED_EVENT_TIME;
|
||||
|
||||
/// Greatest synced event id
|
||||
static IntCounter* GREATEST_SYNCED_EVENT_ID;
|
||||
|
||||
/// Greatest synced event time
|
||||
static IntCounter* GREATEST_SYNCED_EVENT_TIME;
|
||||
|
||||
/// Latest metastore event id
|
||||
static IntCounter* LATEST_EVENT_ID;
|
||||
|
||||
@@ -153,6 +159,12 @@ class MetastoreEventMetrics {
|
||||
|
||||
/// Metric name for the outstanding event count
|
||||
static std::string OUTSTANDING_EVENT_COUNT_METRIC_NAME;
|
||||
|
||||
/// Metric name for greatest synced event id
|
||||
static std::string GREATEST_SYNCED_EVENT_ID_METRIC_NAME;
|
||||
|
||||
/// Metric name for greatest synced event time
|
||||
static std::string GREATEST_SYNCED_EVENT_TIME_METRIC_NAME;
|
||||
};
|
||||
|
||||
} // namespace impala
|
||||
|
||||
@@ -991,6 +991,16 @@ struct TEventProcessorMetrics {
|
||||
|
||||
// Outstanding event count to process on executors
|
||||
22: optional i64 outstanding_event_count
|
||||
|
||||
// Number of metastore events that are pending synchronization up to and including
|
||||
// the latest_event_id
|
||||
23: optional i64 pending_event_count
|
||||
|
||||
// Greatest synced event id
|
||||
24: optional i64 greatest_synced_event_id
|
||||
|
||||
// Greatest synced event time
|
||||
25: optional i64 greatest_synced_event_time
|
||||
}
|
||||
|
||||
struct TCatalogHmsCacheApiMetrics {
|
||||
|
||||
@@ -3626,7 +3626,7 @@
|
||||
"key": "events-processor.events-received-15min-rate"
|
||||
},
|
||||
{
|
||||
"description": "Last metastore event id that the catalog server processed and synced to",
|
||||
"description": "It is the id of the latest metastore event such that all events with id less than or equal to the latest metastore event are definitely processed and synced",
|
||||
"contexts": [
|
||||
"CATALOGSERVER"
|
||||
],
|
||||
@@ -3636,7 +3636,7 @@
|
||||
"key" : "events-processor.last-synced-event-id"
|
||||
},
|
||||
{
|
||||
"description": "Last metastore event time that the catalog server processed and synced to",
|
||||
"description": "It is the time of the latest metastore event such that all events with id less than or equal to the latest metastore event are definitely processed and synced",
|
||||
"contexts": [
|
||||
"CATALOGSERVER"
|
||||
],
|
||||
@@ -3645,6 +3645,26 @@
|
||||
"kind" : "COUNTER",
|
||||
"key" : "events-processor.last-synced-event-time"
|
||||
},
|
||||
{
|
||||
"description": "Greatest synced event id is the id of the latest metastore event such that all events with id less than or equal to the latest metastore event are definitely processed and synced. It is same as last-synced-event-id",
|
||||
"contexts": [
|
||||
"CATALOGSERVER"
|
||||
],
|
||||
"label": "Greatest Synced Event Id",
|
||||
"units": "NONE",
|
||||
"kind" : "COUNTER",
|
||||
"key" : "events-processor.greatest-synced-event-id"
|
||||
},
|
||||
{
|
||||
"description": "Greatest synced event time is the time of the latest metastore event such that all events with id less than or equal to the latest metastore event are definitely processed and synced. It is same as last-synced-event-time",
|
||||
"contexts": [
|
||||
"CATALOGSERVER"
|
||||
],
|
||||
"label": "Greatest Synced Event Time",
|
||||
"units": "NONE",
|
||||
"kind" : "COUNTER",
|
||||
"key" : "events-processor.greatest-synced-event-time"
|
||||
},
|
||||
{
|
||||
"description": "Latest event id in Hive metastore",
|
||||
"contexts": [
|
||||
@@ -3666,7 +3686,7 @@
|
||||
"key" : "events-processor.latest-event-time"
|
||||
},
|
||||
{
|
||||
"description": "Number of pending events to be synced, i.e. the difference between latest-event-id and last-synced-event-id",
|
||||
"description": "Number of pending events to be synced till the latest-event-id",
|
||||
"contexts": [
|
||||
"CATALOGSERVER"
|
||||
],
|
||||
@@ -3676,7 +3696,7 @@
|
||||
"key" : "events-processor.pending-events"
|
||||
},
|
||||
{
|
||||
"description": "Lag time of the event processing, i.e. the difference between latest-event-time and last-synced-event-time",
|
||||
"description": "Lag time of the event processing, i.e. the difference between latest-event-time and greatest-synced-event-time",
|
||||
"contexts": [
|
||||
"CATALOGSERVER"
|
||||
],
|
||||
|
||||
@@ -73,9 +73,9 @@ import org.apache.impala.catalog.HdfsTable;
|
||||
import org.apache.impala.catalog.Hive3MetastoreShimBase;
|
||||
import org.apache.impala.catalog.MetaStoreClientPool;
|
||||
import org.apache.impala.catalog.MetaStoreClientPool.MetaStoreClient;
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.DerivedMetastoreEvent;
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.DerivedMetastoreEventContext;
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.DerivedMetastoreTableEvent;
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.IgnoredEvent;
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.MetastoreTableEvent;
|
||||
import org.apache.impala.catalog.events.MetastoreEventsProcessor.MetaDataFilter;
|
||||
import org.apache.impala.catalog.events.MetastoreNotificationException;
|
||||
import org.apache.impala.catalog.events.SelfEventContext;
|
||||
@@ -650,13 +650,11 @@ public class MetastoreShim extends Hive3MetastoreShimBase {
|
||||
* dummy implementation class defined in this file becomes actual implementation. Need
|
||||
* to change when CommitTxnEvent implementation is supported with IMPALA-13285.
|
||||
*/
|
||||
public static class PseudoCommitTxnEvent extends MetastoreTableEvent
|
||||
implements DerivedMetastoreEvent {
|
||||
PseudoCommitTxnEvent(CommitTxnEvent actualEvent, String dbName, String tableName,
|
||||
boolean isPartitioned, boolean isMaterializedView, List<Long> writeIds,
|
||||
List<Partition> partitions) {
|
||||
super(actualEvent.getCatalogOpExecutor(), actualEvent.getMetrics(),
|
||||
actualEvent.getEvent());
|
||||
public static class PseudoCommitTxnEvent extends DerivedMetastoreTableEvent {
|
||||
PseudoCommitTxnEvent(DerivedMetastoreEventContext context, String dbName,
|
||||
String tableName, boolean isPartitioned, boolean isMaterializedView,
|
||||
List<Long> writeIds, List<Partition> partitions) {
|
||||
super(context);
|
||||
throw new UnsupportedOperationException("PseudoCommitTxnEvent is not supported.");
|
||||
}
|
||||
|
||||
|
||||
@@ -108,7 +108,8 @@ import org.apache.impala.catalog.MetaStoreClientPool.MetaStoreClient;
|
||||
import org.apache.impala.catalog.TableNotFoundException;
|
||||
import org.apache.impala.catalog.TableNotLoadedException;
|
||||
import org.apache.impala.catalog.TableWriteId;
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.DerivedMetastoreEvent;
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.DerivedMetastoreEventContext;
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.DerivedMetastoreTableEvent;
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEvent;
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEventPropertyKey;
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEventType;
|
||||
@@ -896,6 +897,36 @@ public class MetastoreShim extends Hive3MetastoreShimBase {
|
||||
return new CatalogMetastoreServer(catalogOpExecutor);
|
||||
}
|
||||
|
||||
/**
|
||||
* Write info for a table
|
||||
*/
|
||||
private static class AcidTableWriteInfo {
|
||||
private final Table table_;
|
||||
private final List<Long> writeIds_ = new ArrayList<>();
|
||||
private final List<Partition> partitions_ = new ArrayList<>();
|
||||
|
||||
private AcidTableWriteInfo(Table table) {
|
||||
table_ = table;
|
||||
}
|
||||
|
||||
private void appendWriteIdAndPartition(Long writeId, Partition partition) {
|
||||
writeIds_.add(writeId);
|
||||
partitions_.add(partition);
|
||||
}
|
||||
|
||||
private Table getTable() {
|
||||
return table_;
|
||||
}
|
||||
|
||||
private List<Long> getWriteIdList() {
|
||||
return writeIds_;
|
||||
}
|
||||
|
||||
private List<Partition> getPartitionList() {
|
||||
return partitions_;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Metastore event handler for COMMIT_TXN events. Handles commit event for transactional
|
||||
* tables.
|
||||
@@ -990,9 +1021,8 @@ public class MetastoreShim extends Hive3MetastoreShimBase {
|
||||
List<WriteEventInfo> writeEventInfoList) throws Exception {
|
||||
List<Long> writeIds = writeEventInfoList.stream().map(WriteEventInfo::getWriteId)
|
||||
.collect(Collectors.toList());
|
||||
List<Partition> parts = new ArrayList<>();
|
||||
// To load partitions together for the same table, indexes are grouped by table name
|
||||
Map<TableName, Pair<Table, List<Integer>>> tableNameToIdxs = new HashMap<>();
|
||||
Map<TableName, AcidTableWriteInfo> tableNameToWriteInfos = new HashMap<>();
|
||||
for (int i = 0; i < writeIds.size(); i++) {
|
||||
Table tbl = (Table) MessageBuilder.getTObj(
|
||||
writeEventInfoList.get(i).getTableObj(), Table.class);
|
||||
@@ -1008,24 +1038,19 @@ public class MetastoreShim extends Hive3MetastoreShimBase {
|
||||
if (writeEventInfoList.get(i).getPartitionObj() != null) {
|
||||
partition = (Partition) MessageBuilder.getTObj(
|
||||
writeEventInfoList.get(i).getPartitionObj(), Partition.class);
|
||||
Preconditions.checkNotNull(partition);
|
||||
}
|
||||
parts.add(partition);
|
||||
Pair<Table, List<Integer>> pair = tableNameToIdxs.computeIfAbsent(tableName,
|
||||
k -> new Pair<>(tbl, new ArrayList<>()));
|
||||
pair.getSecond().add(i);
|
||||
AcidTableWriteInfo writeInfo = tableNameToWriteInfos.computeIfAbsent(tableName,
|
||||
k -> new AcidTableWriteInfo(tbl));
|
||||
writeInfo.appendWriteIdAndPartition(writeIds.get(i), partition);
|
||||
tableNames_.add(tableName.toString());
|
||||
}
|
||||
for (Map.Entry<TableName, Pair<Table, List<Integer>>> entry :
|
||||
tableNameToIdxs.entrySet()) {
|
||||
Table tbl = entry.getValue().getFirst();
|
||||
List<Long> writeIdsForTable = entry.getValue().getSecond().stream()
|
||||
.map(i -> writeIds.get(i)).collect(Collectors.toList());
|
||||
List<Partition> partsForTable = entry.getValue().getSecond().stream()
|
||||
.map(i -> parts.get(i)).collect(Collectors.toList());
|
||||
for (AcidTableWriteInfo writeInfo : tableNameToWriteInfos.values()) {
|
||||
Table tbl = writeInfo.getTable();
|
||||
addCommittedWriteIdsAndReload(getCatalogOpExecutor(), tbl.getDbName(),
|
||||
tbl.getTableName(), tbl.getPartitionKeysSize() > 0,
|
||||
MetaStoreUtils.isMaterializedViewTable(tbl), writeIdsForTable, partsForTable,
|
||||
getEventId(), getMetrics());
|
||||
MetaStoreUtils.isMaterializedViewTable(tbl), writeInfo.getWriteIdList(),
|
||||
writeInfo.getPartitionList(), getEventId(), getMetrics());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1083,8 +1108,7 @@ public class MetastoreShim extends Hive3MetastoreShimBase {
|
||||
/**
|
||||
* Pseudo commit transaction event handles commit processing for a table
|
||||
*/
|
||||
public static class PseudoCommitTxnEvent extends MetastoreTableEvent
|
||||
implements DerivedMetastoreEvent {
|
||||
public static class PseudoCommitTxnEvent extends DerivedMetastoreTableEvent {
|
||||
private final long txnId_;
|
||||
private final boolean isPartitioned_;
|
||||
private final boolean isMaterializedView_;
|
||||
@@ -1092,12 +1116,12 @@ public class MetastoreShim extends Hive3MetastoreShimBase {
|
||||
private final List<Long> writeIdsInEvent_;
|
||||
private final List<Partition> partitions_;
|
||||
|
||||
PseudoCommitTxnEvent(CommitTxnEvent actualEvent, String dbName, String tableName,
|
||||
boolean isPartitioned, boolean isMaterializedView, List<Long> writeIdsInCatalog,
|
||||
List<Long> writeIdsInEvent, List<Partition> partitions) {
|
||||
super(actualEvent.getCatalogOpExecutor(), actualEvent.getMetrics(),
|
||||
actualEvent.getEvent());
|
||||
txnId_ = actualEvent.txnId_;
|
||||
PseudoCommitTxnEvent(DerivedMetastoreEventContext context, String dbName,
|
||||
String tableName, boolean isPartitioned, boolean isMaterializedView,
|
||||
List<Long> writeIdsInCatalog, List<Long> writeIdsInEvent,
|
||||
List<Partition> partitions) {
|
||||
super(context);
|
||||
txnId_ = ((CommitTxnEvent) context.getActualEvent()).txnId_;
|
||||
writeIdsInCatalog_ = writeIdsInCatalog;
|
||||
writeIdsInEvent_ = writeIdsInEvent;
|
||||
partitions_ = partitions;
|
||||
@@ -1109,8 +1133,15 @@ public class MetastoreShim extends Hive3MetastoreShimBase {
|
||||
|
||||
@Override
|
||||
protected boolean isEventProcessingDisabled() {
|
||||
org.apache.impala.catalog.Table tbl = catalog_.getTableNoThrow(dbName_, tblName_);
|
||||
if (tbl != null && tbl.getCreateEventId() < getEventId()) {
|
||||
msTbl_ = tbl.getMetaStoreTable();
|
||||
}
|
||||
if (msTbl_ == null) {
|
||||
return false;
|
||||
}
|
||||
return super.isEventProcessingDisabled();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected SelfEventContext getSelfEventContext() {
|
||||
@@ -1176,51 +1207,61 @@ public class MetastoreShim extends Hive3MetastoreShimBase {
|
||||
}
|
||||
try {
|
||||
// Build table name to write id indices mapping from the HMS event
|
||||
List<Partition> parts = new ArrayList<>();
|
||||
Map<TableName, Pair<Table, List<Integer>>> tableNameToIdxs = new HashMap<>();
|
||||
Map<TableName, AcidTableWriteInfo> tableNameToWriteInfos = new HashMap<>();
|
||||
int derivedEventCount = 0;
|
||||
DerivedMetastoreEventContext context = new DerivedMetastoreEventContext(event);
|
||||
if (writeEventInfoList != null && !writeEventInfoList.isEmpty()) {
|
||||
List<Long> writeIds = writeEventInfoList.stream().map(WriteEventInfo::getWriteId)
|
||||
.collect(Collectors.toList());
|
||||
for (int i = 0; i < writeIds.size(); i++) {
|
||||
Table tbl = (Table) MessageBuilder.getTObj(
|
||||
writeEventInfoList.get(i).getTableObj(), Table.class);
|
||||
if (event.getCatalogOpExecutor().getCatalog().isHmsEventSyncDisabled(tbl)) {
|
||||
LOG.debug("Not adding write ids to table {}.{} for event {} " +
|
||||
"since table/db level flag {} is set to true",
|
||||
tbl.getDbName(), tbl.getTableName(), event.getEventId(),
|
||||
MetastoreEventPropertyKey.DISABLE_EVENT_HMS_SYNC.getKey());
|
||||
continue;
|
||||
}
|
||||
TableName tableName = new TableName(tbl.getDbName(), tbl.getTableName());
|
||||
Partition partition = null;
|
||||
if (writeEventInfoList.get(i).getPartitionObj() != null) {
|
||||
partition = (Partition) MessageBuilder.getTObj(
|
||||
writeEventInfoList.get(i).getPartitionObj(), Partition.class);
|
||||
Preconditions.checkNotNull(partition);
|
||||
}
|
||||
parts.add(partition);
|
||||
Pair<Table, List<Integer>> pair = tableNameToIdxs.computeIfAbsent(tableName,
|
||||
k -> new Pair<>(tbl, new ArrayList<>()));
|
||||
pair.getSecond().add(i);
|
||||
AcidTableWriteInfo writeInfo = tableNameToWriteInfos.computeIfAbsent(tableName,
|
||||
k -> new AcidTableWriteInfo(tbl));
|
||||
writeInfo.appendWriteIdAndPartition(writeIds.get(i), partition);
|
||||
}
|
||||
// Form list of PseudoCommitTxnEvent
|
||||
List<Long> finalWriteIds = writeIds;
|
||||
for (Map.Entry<TableName, Pair<Table, List<Integer>>> entry :
|
||||
tableNameToIdxs.entrySet()) {
|
||||
for (Map.Entry<TableName, AcidTableWriteInfo> entry :
|
||||
tableNameToWriteInfos.entrySet()) {
|
||||
List<Long> writeIdsInCatalog = tableNameToWriteIds.remove(entry.getKey());
|
||||
Table tbl = entry.getValue().getFirst();
|
||||
AcidTableWriteInfo writeInfo = entry.getValue();
|
||||
Table tbl = writeInfo.getTable();
|
||||
pseudoEvents.add(
|
||||
new PseudoCommitTxnEvent(event, tbl.getDbName(), tbl.getTableName(),
|
||||
new PseudoCommitTxnEvent(context, tbl.getDbName(), tbl.getTableName(),
|
||||
tbl.getPartitionKeysSize() > 0,
|
||||
MetaStoreUtils.isMaterializedViewTable(tbl), writeIdsInCatalog,
|
||||
entry.getValue().getSecond().stream().map(i -> finalWriteIds.get(i))
|
||||
.collect(Collectors.toList()),
|
||||
entry.getValue().getSecond().stream().map(i -> parts.get(i))
|
||||
.collect(Collectors.toList())));
|
||||
writeInfo.getWriteIdList(), writeInfo.getPartitionList()));
|
||||
derivedEventCount++;
|
||||
}
|
||||
}
|
||||
for (Map.Entry<TableName, List<Long>> entry : tableNameToWriteIds.entrySet()) {
|
||||
Table tbl = event.getCatalogOpExecutor().getCatalog()
|
||||
.getTable(entry.getKey().getDb(), entry.getKey().getTbl())
|
||||
.getMetaStoreTable();
|
||||
org.apache.impala.catalog.Table table = event.getCatalogOpExecutor().getCatalog()
|
||||
.getTableNoThrow(entry.getKey().getDb(), entry.getKey().getTbl());
|
||||
if (table != null && table.getMetaStoreTable() != null) {
|
||||
Table tbl = table.getMetaStoreTable();
|
||||
pseudoEvents.add(
|
||||
new PseudoCommitTxnEvent(event, tbl.getDbName(), tbl.getTableName(),
|
||||
new PseudoCommitTxnEvent(context, tbl.getDbName(), tbl.getTableName(),
|
||||
tbl.getPartitionKeysSize() > 0,
|
||||
MetaStoreUtils.isMaterializedViewTable(tbl), entry.getValue(),
|
||||
Collections.emptyList(), Collections.emptyList()));
|
||||
derivedEventCount++;
|
||||
}
|
||||
}
|
||||
context.setDerivedEventsCount(derivedEventCount);
|
||||
} catch (Exception e) {
|
||||
throw new MetastoreNotificationNeedsInvalidateException(String.format(
|
||||
"Failed to form PseudoCommitTxnEvent for txn %d. Event processing cannot " +
|
||||
|
||||
@@ -178,7 +178,7 @@ public abstract class Catalog implements AutoCloseable {
|
||||
* Returns the Table object for the given dbName/tableName or null if the database or
|
||||
* table does not exist.
|
||||
*/
|
||||
public Table getTableNoThrow(String dbName, String tableName) {
|
||||
public @Nullable Table getTableNoThrow(String dbName, String tableName) {
|
||||
Db db = getDb(dbName);
|
||||
if (db == null) return null;
|
||||
return db.getTable(tableName);
|
||||
@@ -220,7 +220,7 @@ public abstract class Catalog implements AutoCloseable {
|
||||
* Removes a table from the catalog and returns the table that was removed, or null
|
||||
* if the table/database does not exist.
|
||||
*/
|
||||
public Table removeTable(TTableName tableName) {
|
||||
public @Nullable Table removeTable(TTableName tableName) {
|
||||
// Remove the old table name from the cache and add the new table.
|
||||
Db db = getDb(tableName.getDb_name());
|
||||
if (db == null) return null;
|
||||
@@ -344,7 +344,7 @@ public abstract class Catalog implements AutoCloseable {
|
||||
* ordered by argument precision (e.g. double is preferred over float) and then by
|
||||
* alphabetical order of argument type name, to guarantee deterministic results.
|
||||
*/
|
||||
public Function getFunction(Function desc, Function.CompareMode mode) {
|
||||
public @Nullable Function getFunction(Function desc, Function.CompareMode mode) {
|
||||
Db db = getDb(desc.dbName());
|
||||
if (db == null) return null;
|
||||
return db.getFunction(desc, mode);
|
||||
@@ -355,7 +355,7 @@ public abstract class Catalog implements AutoCloseable {
|
||||
* the Function object that was removed if the function existed, otherwise returns
|
||||
* null.
|
||||
*/
|
||||
public Function removeFunction(Function desc) {
|
||||
public @Nullable Function removeFunction(Function desc) {
|
||||
Db db = getDb(desc.dbName());
|
||||
if (db == null) return null;
|
||||
return db.removeFunction(desc);
|
||||
|
||||
@@ -2488,8 +2488,6 @@ public class CatalogServiceCatalog extends Catalog {
|
||||
// pause the event processing since the cache is anyways being cleared
|
||||
metastoreEventProcessor_.pause();
|
||||
metastoreEventProcessor_.clear();
|
||||
// Clear delete event log
|
||||
metastoreEventProcessor_.getDeleteEventLog().garbageCollect(currentEventId);
|
||||
// Update the HDFS cache pools
|
||||
try {
|
||||
// We want only 'true' HDFS filesystems to poll the HDFS cache (i.e not S3,
|
||||
@@ -4173,7 +4171,7 @@ public class CatalogServiceCatalog extends Catalog {
|
||||
MetastoreEventsProcessor ep = (MetastoreEventsProcessor) metastoreEventProcessor_;
|
||||
StringBuilder info = new StringBuilder();
|
||||
if (cmdType == MetastoreEventsProcessor.EventProcessorCmdType.PAUSE) {
|
||||
ep.pause();
|
||||
ep.pauseGracefully();
|
||||
} else if (cmdType == MetastoreEventsProcessor.EventProcessorCmdType.START) {
|
||||
if (!startEventProcessorHelper(params, ep, resp, info)) {
|
||||
// 'resp' is updated in startEventProcessorHelper() for errors.
|
||||
@@ -4182,7 +4180,7 @@ public class CatalogServiceCatalog extends Catalog {
|
||||
}
|
||||
info.append(String.format("EventProcessor status: %s. LastSyncedEventId: %d. " +
|
||||
"LatestEventId: %d.",
|
||||
ep.getStatus(), ep.getLastSyncedEventId(), ep.getLatestEventId()));
|
||||
ep.getStatus(), ep.getGreatestSyncedEventId(), ep.getLatestEventId()));
|
||||
resp.setStatus(new TStatus(TErrorCode.OK, Collections.emptyList()));
|
||||
resp.setInfo(info.toString());
|
||||
return resp;
|
||||
@@ -4193,7 +4191,7 @@ public class CatalogServiceCatalog extends Catalog {
|
||||
StringBuilder warnings) {
|
||||
if (!params.isSetEvent_id()) {
|
||||
if (ep.getStatus() != EventProcessorStatus.ACTIVE) {
|
||||
ep.start(ep.getLastSyncedEventId());
|
||||
ep.start(ep.getGreatestSyncedEventId());
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@@ -24,7 +24,6 @@ import java.util.concurrent.atomic.AtomicInteger;
|
||||
import org.apache.impala.catalog.CatalogException;
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.DerivedMetastoreEvent;
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.MetastoreDatabaseEvent;
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEventType;
|
||||
|
||||
/**
|
||||
* An instance of this class is used to synchronize all the TableProcessors to process
|
||||
@@ -41,12 +40,12 @@ import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEventType;
|
||||
* indicates that to DbProcessor event processing with {@link DbBarrierEvent#proceed()}
|
||||
* and do not process any further events beyond it. On subsequent event processing
|
||||
* schedules, TableProcessor's event processing determines whether database event is
|
||||
* processed on DbProcessor with {@link DbBarrierEvent#isProcessed()}.
|
||||
* processed on DbProcessor with {@link DbBarrierEvent#isAllDerivedEventsProcessed()}.
|
||||
* <p>
|
||||
* Once all the TableProcessors indicate DbProcessor event processing, DbProcessor
|
||||
* process the DbBarrierEvent and marks it as processed. And the TableProcessors detects
|
||||
* the event status with {@link DbBarrierEvent#isProcessed()}. Henceforth, all the
|
||||
* TableProcessors continue to process their table events independently.
|
||||
* the event status with {@link DbBarrierEvent#isAllDerivedEventsProcessed()}. Henceforth,
|
||||
* all the TableProcessors continue to process their table events independently.
|
||||
*
|
||||
* @see org.apache.impala.catalog.events.DbEventExecutor.DbProcessor
|
||||
* @see org.apache.impala.catalog.events.TableEventExecutor.TableProcessor
|
||||
@@ -74,10 +73,6 @@ public class DbBarrierEvent extends MetastoreDatabaseEvent
|
||||
public void processIfEnabled() throws CatalogException, MetastoreNotificationException {
|
||||
Preconditions.checkState(expectedProceedCount_.get() == 0);
|
||||
actualEvent_.processIfEnabled();
|
||||
if (getEventType() == MetastoreEventType.DROP_DATABASE) {
|
||||
catalog_.getMetastoreEventProcessor().getDeleteEventLog().removeEvent(getEventId());
|
||||
}
|
||||
isProcessed_ = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -95,21 +90,13 @@ public class DbBarrierEvent extends MetastoreDatabaseEvent
|
||||
* <p>
|
||||
* TableProcessor invokes it when this event is seen during event processing. And
|
||||
* waits for DbProcessor to process the event. TableProcessor determines whether
|
||||
* the event is processed at DbProcessor using {@link DbBarrierEvent#isProcessed()}
|
||||
* method.
|
||||
* the event is processed at DbProcessor using
|
||||
* {@link DbBarrierEvent#isAllDerivedEventsProcessed()} method.
|
||||
*/
|
||||
void proceed() {
|
||||
decrExpectedProceedCount();
|
||||
}
|
||||
|
||||
/**
|
||||
* Used to determine if this event is processed on DbProcessor.
|
||||
* @return True if the event is processed. False otherwise
|
||||
*/
|
||||
boolean isProcessed() {
|
||||
return isProcessed_;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the count of TableProcessor that are expected to reach this event during
|
||||
* event processing.
|
||||
@@ -136,4 +123,31 @@ public class DbBarrierEvent extends MetastoreDatabaseEvent
|
||||
int value = expectedProceedCount_.decrementAndGet();
|
||||
debugLog("Number of table processors expected to process the event: {}", value);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the actual metastore event
|
||||
* @return Actual metastore event
|
||||
*/
|
||||
@Override
|
||||
public MetastoreEvents.MetastoreEvent getActualEvent() {
|
||||
return actualEvent_;
|
||||
}
|
||||
|
||||
/**
|
||||
* Marks the event as processed.
|
||||
*/
|
||||
@Override
|
||||
public void markProcessed() {
|
||||
Preconditions.checkState(expectedProceedCount_.get() == 0);
|
||||
isProcessed_ = true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines whether the event is processed on DbProcessor.
|
||||
* @return True if the event is processed. False otherwise
|
||||
*/
|
||||
@Override
|
||||
public boolean isAllDerivedEventsProcessed() {
|
||||
return isProcessed_;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -39,8 +39,8 @@ import org.apache.impala.catalog.events.MetastoreEvents.DropTableEvent;
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.MetastoreDatabaseEvent;
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEvent;
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEventType;
|
||||
import org.apache.impala.catalog.events.MetastoreEventsProcessor.EventProcessorStatus;
|
||||
import org.apache.impala.catalog.events.TableEventExecutor.TableProcessor;
|
||||
import org.apache.impala.common.PrintUtils;
|
||||
import org.apache.impala.service.BackendConfig;
|
||||
import org.apache.impala.util.ClassUtil;
|
||||
import org.apache.impala.util.ThreadNameAnnotator;
|
||||
@@ -138,8 +138,14 @@ public class DbEventExecutor {
|
||||
* Indicates whether DbProcessor is terminating. Events are dispatched and processed
|
||||
* only when isTerminating_ is false.
|
||||
* <p>
|
||||
* Event dispatching and processing (i.e., {@link DbProcessor#process()}) and clearing
|
||||
* DbProcessor event queueing {@link DbProcessor#enqueue(MetastoreEvent)},
|
||||
* event dispatching and processing (i.e., {@link DbProcessor#process()}) and clearing
|
||||
* (i.e., {@link DbProcessor#clear()}) are invoked from different threads.
|
||||
* <p>
|
||||
* {@link DbProcessor#enqueue(MetastoreEvent)} acquires processorLock_ to atomically
|
||||
* update the event queue, increment outstanding event count, so that
|
||||
* {@link DbProcessor#process()} gets the consistent view of them together.
|
||||
* <p>
|
||||
* {@link DbProcessor#clear()} acquires processorLock_ to immediately set
|
||||
* isTerminating_ to true, thereby preventing any further event dispatching or
|
||||
* processing.
|
||||
@@ -164,13 +170,12 @@ public class DbEventExecutor {
|
||||
* or {@link DbEventExecutor#stop()} is invoked, all associated DbProcessors are
|
||||
* forcibly cleared and removed.
|
||||
* <p>
|
||||
* Lock contention is unlikely at the most frequent call sites (i.e., within
|
||||
* {@link DbProcessor#process()} and its related methods), as clear() is called
|
||||
* infrequently.
|
||||
* <p>
|
||||
* There is no lock contention between the {@link DbEventExecutor#stop()} and
|
||||
* {@link DbEventExecutor#process()} flows, since scheduling of process() is halted
|
||||
* by stop().
|
||||
* by stop(). Also, there is no lock contention between the
|
||||
* {@link DbEventExecutor#enqueue(MetastoreEvent)}, {@link DbEventExecutor#cleanup()},
|
||||
* {@link DbEventExecutor#clear()} and {@link DbEventExecutor#stop()} because
|
||||
* {@link EventExecutorService} invokes these methods in a synchronized manner.
|
||||
*/
|
||||
private final Object processorLock_ = new Object();
|
||||
private boolean isTerminating_ = false;
|
||||
@@ -180,6 +185,14 @@ public class DbEventExecutor {
|
||||
dbName_ = dbName;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets Metastore event processor
|
||||
* @return
|
||||
*/
|
||||
private MetastoreEventsProcessor getEventProcessor() {
|
||||
return dbEventExecutor_.eventProcessor_;
|
||||
}
|
||||
|
||||
/**
|
||||
* Queue event to the DbProcessor for processing.
|
||||
* @param event Metastore event
|
||||
@@ -189,8 +202,11 @@ public class DbEventExecutor {
|
||||
if (event.getEventType() == MetastoreEventType.DROP_DATABASE) {
|
||||
skipEventId_.set(event.getEventId());
|
||||
}
|
||||
synchronized (processorLock_) {
|
||||
Preconditions.checkState(!isTerminating());
|
||||
inEvents_.offer(event);
|
||||
dbEventExecutor_.incrOutstandingEventCount();
|
||||
}
|
||||
event.debugLog("Enqueued for db: {} on executor: {}", dbName_,
|
||||
dbEventExecutor_.name_);
|
||||
}
|
||||
@@ -220,12 +236,10 @@ public class DbEventExecutor {
|
||||
TableProcessor tableProcessor =
|
||||
tableEventExecutor.getOrCreateTableProcessor(fqTableName);
|
||||
tableProcessors_.add(tableProcessor);
|
||||
if (tableProcessor.isEmpty()) {
|
||||
// Prepend all the outstanding db barrier events to TableProcessor so that they
|
||||
// do not process the table events received before these db barrier events are
|
||||
// processed
|
||||
barrierEvents_.forEach(tableProcessor::enqueue);
|
||||
}
|
||||
if (tableProcessor.isEmpty()) barrierEvents_.forEach(tableProcessor::enqueue);
|
||||
tableProcessor.enqueue(event);
|
||||
}
|
||||
}
|
||||
@@ -241,9 +255,7 @@ public class DbEventExecutor {
|
||||
if (isTerminating()) return;
|
||||
barrierEvent = new DbBarrierEvent(event);
|
||||
tableProcessors_.forEach(tableProcessor -> {
|
||||
if (!tableProcessor.isEmpty()) {
|
||||
tableProcessor.enqueue(barrierEvent);
|
||||
}
|
||||
if (!tableProcessor.isEmpty()) tableProcessor.enqueue(barrierEvent);
|
||||
});
|
||||
barrierEvents_.offer(barrierEvent);
|
||||
dbEventExecutor_.incrOutstandingEventCount();
|
||||
@@ -251,21 +263,36 @@ public class DbEventExecutor {
|
||||
}
|
||||
|
||||
/**
|
||||
* Process the database events
|
||||
* Method invoked after processing the event.
|
||||
* @param event Db barrier event
|
||||
*/
|
||||
private void postProcessEvent(DbBarrierEvent event) {
|
||||
MetastoreEventsProcessor eventProcessor = getEventProcessor();
|
||||
event.markProcessed();
|
||||
eventProcessor.getEventExecutorService()
|
||||
.removeFromInProgressLog(event.getEventId());
|
||||
if (event.getEventType() == MetastoreEventType.DROP_DATABASE) {
|
||||
eventProcessor.getDeleteEventLog().removeEvent(event.getEventId());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Process the database event
|
||||
* @param barrierEvent Db Barrier event
|
||||
* @return True if event is processed. False otherwise
|
||||
* @throws Exception
|
||||
*/
|
||||
private void processDbEvents() throws Exception {
|
||||
DbBarrierEvent barrierEvent;
|
||||
String annotation = "Processing %s for db: " + dbName_;
|
||||
while ((barrierEvent = barrierEvents_.peek()) != null) {
|
||||
if (isTerminating()) return;
|
||||
if (barrierEvent.getExpectedProceedCount() != 0) {
|
||||
return;
|
||||
}
|
||||
private boolean processDbEvent(DbBarrierEvent barrierEvent) throws Exception {
|
||||
if (isTerminating()) return false;
|
||||
if (barrierEvent.getExpectedProceedCount() != 0) return false;
|
||||
try (ThreadNameAnnotator tna = new ThreadNameAnnotator(
|
||||
String.format(annotation, barrierEvent.getEventDesc()))) {
|
||||
String.format("Processing %s for db: %s", barrierEvent.getEventDesc(),
|
||||
dbName_))) {
|
||||
long processingStartTime = System.currentTimeMillis();
|
||||
barrierEvent.processIfEnabled();
|
||||
barrierEvent.infoLog("Processed for db: {}", dbName_);
|
||||
barrierEvent.infoLog("Scheduling delay: {}, Process time: {}",
|
||||
PrintUtils.printTimeMs(processingStartTime - barrierEvent.getDispatchTime()),
|
||||
PrintUtils.printTimeMs(System.currentTimeMillis() - processingStartTime));
|
||||
} catch (Exception processingEx) {
|
||||
try {
|
||||
if (!barrierEvent.onFailure(processingEx)) {
|
||||
@@ -277,11 +304,34 @@ public class DbEventExecutor {
|
||||
throw processingEx;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Process the database events
|
||||
* @throws EventProcessException
|
||||
*/
|
||||
private void processDbEvents() throws EventProcessException {
|
||||
DbBarrierEvent barrierEvent;
|
||||
while ((barrierEvent = barrierEvents_.peek()) != null) {
|
||||
try {
|
||||
boolean isProcessed = processDbEvent(barrierEvent);
|
||||
if (!isProcessed) return;
|
||||
postProcessEvent(barrierEvent);
|
||||
synchronized (processorLock_) {
|
||||
if (isTerminating()) return;
|
||||
Preconditions.checkState(barrierEvents_.poll() == barrierEvent);
|
||||
dbEventExecutor_.decrOutstandingEventCount(1);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
// Throwing EventProcessException triggers global invalidates metadata without
|
||||
// user intervention iff invalidate_global_metadata_on_event_processing_failure
|
||||
// flag is true. Otherwise, user has to explicitly issue invalidate metadata.
|
||||
// Invalidate metadata resets catalog instance that clears EventExecutorService.
|
||||
// And EventExecutorService inherently clears all DbEventExecutor and
|
||||
// TableEventExecutor thereby removing all DbProcessors and TableProcessors.
|
||||
throw new EventProcessException(barrierEvent.getEvent(), e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -300,7 +350,6 @@ public class DbEventExecutor {
|
||||
dbEventExecutor_.decrOutstandingEventCount(barrierEvents_.size());
|
||||
inEvents_.clear();
|
||||
barrierEvents_.clear();
|
||||
skipEventId_.set(-1);
|
||||
lastEventQueuedTime_ = 0;
|
||||
cleanIfNecessary(true);
|
||||
Preconditions.checkState(tableProcessors_.isEmpty());
|
||||
@@ -344,12 +393,10 @@ public class DbEventExecutor {
|
||||
/**
|
||||
* Skip the metastore event from processing if possible.
|
||||
* @param event Metastore event
|
||||
* @param dropDbEventId Drop database event id if drop database event is queued for
|
||||
* processing. Else -1.
|
||||
* @return True if event is skipped. Else false.
|
||||
*/
|
||||
private boolean skipEventIfPossible(MetastoreEvent event, long dropDbEventId) {
|
||||
if (event.getEventId() >= dropDbEventId || event instanceof DropTableEvent ||
|
||||
private boolean skipEventIfPossible(MetastoreEvent event) {
|
||||
if (event.getEventId() >= skipEventId_.get() || event instanceof DropTableEvent ||
|
||||
event instanceof RenameTableBarrierEvent) {
|
||||
return false;
|
||||
}
|
||||
@@ -357,9 +404,35 @@ public class DbEventExecutor {
|
||||
.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC);
|
||||
eventSkipCounter.inc();
|
||||
event.debugLog("Incremented skipped metric to {}", eventSkipCounter.getCount());
|
||||
MetastoreEventsProcessor eventProcessor = getEventProcessor();
|
||||
eventProcessor.getEventExecutorService()
|
||||
.removeFromInProgressLog(event.getEventId());
|
||||
if (event.isDropEvent()) {
|
||||
dbEventExecutor_.eventProcessor_.getDeleteEventLog()
|
||||
.removeEvent(event.getEventId());
|
||||
eventProcessor.getDeleteEventLog().removeEvent(event.getEventId());
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Dispatches the table events and database events to the TableProcessors that needs
|
||||
* to process them.
|
||||
* @param event Metastore event
|
||||
* @return True if event is dispatched. False otherwise
|
||||
*/
|
||||
private boolean dispatchEvent(MetastoreEvent event) {
|
||||
if (isTerminating()) return false;
|
||||
MetastoreEventsProcessor eventProcessor = getEventProcessor();
|
||||
if (!eventProcessor.canProcessEventInCurrentStatus()) {
|
||||
LOG.warn("Event processing is skipped for executor: {} since status is {}",
|
||||
dbEventExecutor_.name_, eventProcessor.getStatus());
|
||||
return false;
|
||||
}
|
||||
if (!skipEventIfPossible(event)) {
|
||||
if (event.isDatabaseEvent()) {
|
||||
dispatchDbEvent((MetastoreDatabaseEvent) event);
|
||||
} else {
|
||||
dispatchTableEvent(event);
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
@@ -369,34 +442,30 @@ public class DbEventExecutor {
|
||||
* periodically to process events for the DbProcessor. It dispatches table events and
|
||||
* database events to the TableProcessors that needs to process them and also process
|
||||
* the database events that are eligible for processing.
|
||||
* @throws Exception
|
||||
* @throws EventProcessException
|
||||
*/
|
||||
private void process() throws Exception {
|
||||
private void process() throws EventProcessException {
|
||||
MetastoreEvent event;
|
||||
long skipEventId = skipEventId_.get();
|
||||
while ((event = inEvents_.peek()) != null) {
|
||||
if (isTerminating()) return;
|
||||
if (dbEventExecutor_.eventProcessor_.getStatus() != EventProcessorStatus.ACTIVE) {
|
||||
LOG.warn("Event processing is skipped for executor: {} since status is {}",
|
||||
dbEventExecutor_.name_,
|
||||
dbEventExecutor_.eventProcessor_.getStatus());
|
||||
return;
|
||||
}
|
||||
if (!skipEventIfPossible(event, skipEventId)) {
|
||||
if (event.isDatabaseEvent()) {
|
||||
dispatchDbEvent((MetastoreDatabaseEvent) event);
|
||||
} else {
|
||||
dispatchTableEvent(event);
|
||||
}
|
||||
}
|
||||
try {
|
||||
boolean isDispatched = dispatchEvent(event);
|
||||
if (!isDispatched) return;
|
||||
synchronized (processorLock_) {
|
||||
if (isTerminating()) return;
|
||||
Preconditions.checkState(inEvents_.poll() == event);
|
||||
dbEventExecutor_.decrOutstandingEventCount(1);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
// Throwing EventProcessException triggers global invalidates metadata without
|
||||
// user intervention iff invalidate_global_metadata_on_event_processing_failure
|
||||
// flag is true. Otherwise, user has to explicitly issue invalidate metadata.
|
||||
// Invalidate metadata resets catalog instance that clears EventExecutorService.
|
||||
// And EventExecutorService inherently clears all DbEventExecutor and
|
||||
// TableEventExecutor thereby removing all DbProcessors and TableProcessors.
|
||||
throw new EventProcessException(event.getEvent(), e);
|
||||
}
|
||||
}
|
||||
processDbEvents();
|
||||
skipEventId_.compareAndSet(skipEventId, -1);
|
||||
cleanIfNecessary(false);
|
||||
}
|
||||
}
|
||||
@@ -499,9 +568,7 @@ public class DbEventExecutor {
|
||||
Iterator<Map.Entry<String, DbProcessor>> it = dbProcessors_.entrySet().iterator();
|
||||
while (it.hasNext()) {
|
||||
DbProcessor dbProcessor = it.next().getValue();
|
||||
if (force) {
|
||||
dbProcessor.clear();
|
||||
}
|
||||
if (force) dbProcessor.clear();
|
||||
if (dbProcessor.canBeRemoved()) {
|
||||
unAssignEventExecutor(dbProcessor.dbName_);
|
||||
it.remove();
|
||||
@@ -550,7 +617,7 @@ public class DbEventExecutor {
|
||||
* @param event Metastore event
|
||||
*/
|
||||
void enqueue(MetastoreEvent event) {
|
||||
if (eventProcessor_.getStatus() != EventProcessorStatus.ACTIVE) {
|
||||
if (!eventProcessor_.canProcessEventInCurrentStatus()) {
|
||||
event.warnLog("Event is not queued to executor: {} since status is {}", name_,
|
||||
eventProcessor_.getStatus());
|
||||
return;
|
||||
@@ -581,9 +648,7 @@ public class DbEventExecutor {
|
||||
private TableEventExecutor getOrAssignTableEventExecutor(String fqTableName) {
|
||||
Preconditions.checkNotNull(fqTableName);
|
||||
TableEventExecutor executor = tableToEventExecutor_.get(fqTableName);
|
||||
if (executor != null) {
|
||||
return executor;
|
||||
}
|
||||
if (executor != null) return executor;
|
||||
long minOutStandingEvents = Long.MAX_VALUE;
|
||||
long minTableCount = Long.MAX_VALUE;
|
||||
for (TableEventExecutor tee : tableEventExecutors_) {
|
||||
@@ -627,13 +692,12 @@ public class DbEventExecutor {
|
||||
try {
|
||||
for (Map.Entry<String, DbProcessor> entry : dbProcessors_.entrySet()) {
|
||||
DbProcessor dbProcessor = entry.getValue();
|
||||
if (eventProcessor_.getStatus() != EventProcessorStatus.ACTIVE) {
|
||||
break;
|
||||
}
|
||||
if (!eventProcessor_.canProcessEventInCurrentStatus()) break;
|
||||
dbProcessor.process();
|
||||
}
|
||||
} catch (Exception e) {
|
||||
eventProcessor_.handleEventProcessException(e);
|
||||
} catch (EventProcessException e) {
|
||||
LOG.error("Exception occurred for executor: {}", name_);
|
||||
eventProcessor_.handleEventProcessException(e.getException(), e.getEvent());
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -22,9 +22,11 @@ import com.google.common.base.Preconditions;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.TreeMap;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
@@ -34,13 +36,18 @@ import org.apache.impala.catalog.TableWriteId;
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.AbortTxnEvent;
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.AlterTableEvent;
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.CreateTableEvent;
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.DerivedMetastoreEventContext;
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.DropTableEvent;
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.IgnoredEvent;
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEvent;
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEventType;
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.MetastoreTableEvent;
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.PseudoAbortTxnEvent;
|
||||
import org.apache.impala.common.Pair;
|
||||
import org.apache.impala.common.PrintUtils;
|
||||
import org.apache.impala.compat.MetastoreShim;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* This class provides the necessary methods to init, start, clear, shutdown the
|
||||
@@ -51,6 +58,8 @@ import org.apache.impala.compat.MetastoreShim;
|
||||
* {@link #shutdown(boolean graceful)} is invoked, cannot be started or dispatch events.
|
||||
*/
|
||||
public class EventExecutorService {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(EventExecutorService.class);
|
||||
|
||||
// DbEventExecutor and TableEventExecutor threads task schedule interval
|
||||
private static final int EXECUTOR_SCHEDULE_INTERVAL_MS = 10;
|
||||
|
||||
@@ -69,6 +78,22 @@ public class EventExecutorService {
|
||||
// List of DbEventExecutor
|
||||
private final List<DbEventExecutor> dbEventExecutors_;
|
||||
|
||||
// In-progress event log. Maintains the metastore events waiting to be processed on
|
||||
// executors. It is a map of metastore event id to the metastore event. Metastore event
|
||||
// can be a delimiter or non-delimiter event. Delimiter is a kind of metastore event
|
||||
// that do not require event processing. Delimeter event can be:
|
||||
// 1. A CommitTxnEvent that do not have any write event info for a given transaction.
|
||||
// 2. An AbortTxnEvent that do not have write ids for a given transaction.
|
||||
// 3. An IgnoredEvent.
|
||||
// They are not queued to DbEventExecutor for processing. They are just maintained in
|
||||
// the inProgressLog_ to preserve continuity and correctness in synchronization
|
||||
// tracking. The delimiter events are removed from this map when their preceding
|
||||
//non-delimiter metastore event is removed.
|
||||
private final TreeMap<Long, MetastoreEvent> inProgressLog_ = new TreeMap<>();
|
||||
|
||||
// Processed event log. Maintains event id to HMS event notification time mapping.
|
||||
private TreeMap<Long, Long> processedLog_ = new TreeMap<>();
|
||||
|
||||
/**
|
||||
* Database name to DbEventExecutor map. This map is mutated only by
|
||||
* DbEventExecutor through
|
||||
@@ -92,7 +117,7 @@ public class EventExecutorService {
|
||||
}
|
||||
|
||||
/**
|
||||
* This method is only for testing.
|
||||
* This method is to get DbEventExecutor list. It is only for testing.
|
||||
* @return List of DbEventExecutors it holds.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
@@ -101,30 +126,63 @@ public class EventExecutorService {
|
||||
}
|
||||
|
||||
/**
|
||||
* This method is only for testing and must not be used anywhere else.
|
||||
* This method is to set the event executor status without actually starting it. It is
|
||||
* only for testing and must not be used anywhere else.
|
||||
* @param status EventExecutorStatus
|
||||
*/
|
||||
@VisibleForTesting
|
||||
public void setStatus(EventExecutorStatus status) {
|
||||
void setStatus(EventExecutorStatus status) {
|
||||
Preconditions.checkState(status_ != EventExecutorStatus.STOPPED);
|
||||
if (status == EventExecutorStatus.ACTIVE) {
|
||||
status_ = EventExecutorStatus.ACTIVE;
|
||||
makeActive();
|
||||
} else if (status == EventExecutorStatus.STOPPED) {
|
||||
shutdown(true);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This method is to get in-progress log. It is only for testing.
|
||||
* @return in-progress log tree map.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
TreeMap<Long, MetastoreEvent> getInProgressLog() {
|
||||
return inProgressLog_;
|
||||
}
|
||||
|
||||
/**
|
||||
* This method is to get processed log. It is only for testing.
|
||||
* @return processed log tree map
|
||||
*/
|
||||
@VisibleForTesting
|
||||
TreeMap<Long, Long> getProcessedLog() {
|
||||
return processedLog_;
|
||||
}
|
||||
|
||||
/**
|
||||
* Method to make the EventExecutorService active. This method is invoked from
|
||||
* {@link #start()}.
|
||||
*/
|
||||
private synchronized void makeActive() {
|
||||
clearLogs();
|
||||
// Event processor set its lastSyncedEventId to the event id from which event
|
||||
// processing must start. Event processor start is a synchronized method and this
|
||||
// method gets through that flow.
|
||||
addToProcessedLog(eventProcessor_.getLastSyncedEventId(),
|
||||
eventProcessor_.getLastSyncedEventTime());
|
||||
status_ = EventExecutorStatus.ACTIVE;
|
||||
}
|
||||
|
||||
/**
|
||||
* Start the EventExecutorService. It has to be started to make it ready to process the
|
||||
* events. It is invoked from {@link ExternalEventsProcessor#start()}.
|
||||
* Once shutdown, it cannot be started.
|
||||
* events. It is invoked from {@link ExternalEventsProcessor#start()} and
|
||||
* {@link ExternalEventsProcessor#start(long)}. Once shutdown, it cannot be started.
|
||||
*/
|
||||
synchronized void start() {
|
||||
Preconditions.checkState(status_ != EventExecutorStatus.STOPPED);
|
||||
if (status_ == EventExecutorStatus.INACTIVE) {
|
||||
dbEventExecutors_.forEach(DbEventExecutor::start);
|
||||
status_ = EventExecutorStatus.ACTIVE;
|
||||
}
|
||||
makeActive();
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -134,6 +192,7 @@ public class EventExecutorService {
|
||||
*/
|
||||
synchronized void clear() {
|
||||
dbEventExecutors_.parallelStream().forEach(DbEventExecutor::clear);
|
||||
clearLogs();
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -148,6 +207,15 @@ public class EventExecutorService {
|
||||
eventProcessor_.ensureEventsProcessedInHierarchicalMode(3600000);
|
||||
}
|
||||
dbEventExecutors_.parallelStream().forEach(DbEventExecutor::stop);
|
||||
clearLogs();
|
||||
}
|
||||
|
||||
/**
|
||||
* Method to clear in-progress and processed event logs.
|
||||
*/
|
||||
private synchronized void clearLogs() {
|
||||
inProgressLog_.clear();
|
||||
processedLog_.clear();
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -160,12 +228,34 @@ public class EventExecutorService {
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the aggregated outstanding event count of all the DbEventExecutor instances.
|
||||
* Returns the number of outstanding events currently queued for processing.
|
||||
* @return Outstanding event count
|
||||
*/
|
||||
long getOutstandingEventCount() {
|
||||
return dbEventExecutors_.stream().mapToLong(DbEventExecutor::getOutstandingEventCount)
|
||||
.sum();
|
||||
synchronized long getOutstandingEventCount() {
|
||||
return inProgressLog_.size();
|
||||
}
|
||||
|
||||
/**
|
||||
* Calculates the number of events that must be synchronized in order to ensure that
|
||||
* the greatest synced event id reaches the specified {@code eventId}.
|
||||
* @param eventId Target event id to which the greatest synced event id should advance
|
||||
* @return Estimated number of events that need to be synchronized
|
||||
*/
|
||||
synchronized long getPendingEventCount(long eventId) {
|
||||
Preconditions.checkState(status_ == EventExecutorStatus.ACTIVE,
|
||||
"EventExecutorService must be active");
|
||||
long greatestSyncedEventId = processedLog_.firstKey();
|
||||
if (eventId <= greatestSyncedEventId) return 0;
|
||||
if (inProgressLog_.isEmpty()) return eventId - greatestSyncedEventId;
|
||||
long lastUnprocessedEventId = inProgressLog_.lastKey();
|
||||
if (eventId <= lastUnprocessedEventId) {
|
||||
return inProgressLog_.headMap(eventId, true).size();
|
||||
}
|
||||
long closerEventId = processedLog_.floorKey(eventId);
|
||||
if (closerEventId == greatestSyncedEventId) {
|
||||
return eventId - lastUnprocessedEventId + inProgressLog_.size();
|
||||
}
|
||||
return eventId - closerEventId + inProgressLog_.headMap(closerEventId).size();
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -200,27 +290,32 @@ public class EventExecutorService {
|
||||
Preconditions.checkState(status_ == EventExecutorStatus.ACTIVE,
|
||||
"EventExecutorService must be active");
|
||||
Preconditions.checkNotNull(event);
|
||||
boolean isEventDispatched = false;
|
||||
if (event instanceof MetastoreShim.CommitTxnEvent) {
|
||||
processCommitTxnEvent((MetastoreShim.CommitTxnEvent) event);
|
||||
isEventDispatched = processCommitTxnEvent((MetastoreShim.CommitTxnEvent) event);
|
||||
} else if (event instanceof AbortTxnEvent) {
|
||||
processAbortTxnEvent((AbortTxnEvent) event);
|
||||
isEventDispatched = processAbortTxnEvent((AbortTxnEvent) event);
|
||||
} else if (event instanceof AlterTableEvent && ((AlterTableEvent) event).isRename()) {
|
||||
processAlterTableRenameEvent((AlterTableEvent) event);
|
||||
isEventDispatched = true;
|
||||
} else if (event instanceof IgnoredEvent) {
|
||||
event.debugLog("Ignoring event type {}", event.getEvent().getEventType());
|
||||
} else {
|
||||
DbEventExecutor dbEventExecutor = getOrFindDbEventExecutor(event.getDbName());
|
||||
dbEventExecutor.enqueue(event);
|
||||
isEventDispatched = true;
|
||||
}
|
||||
addToInProgressLog(event, !isEventDispatched);
|
||||
}
|
||||
|
||||
/**
|
||||
* Dispatches pseudo commit transaction events to the tables involved in transaction for
|
||||
* processing.
|
||||
* @param commitTxnEvent Commit transaction event
|
||||
* @return True if any pseudo-events are dispatched. False otherwise.
|
||||
* @throws MetastoreNotificationException
|
||||
*/
|
||||
private void processCommitTxnEvent(MetastoreShim.CommitTxnEvent commitTxnEvent)
|
||||
private boolean processCommitTxnEvent(MetastoreShim.CommitTxnEvent commitTxnEvent)
|
||||
throws MetastoreNotificationException {
|
||||
List<MetastoreShim.PseudoCommitTxnEvent> pseudoCommitTxnEvents =
|
||||
MetastoreShim.getPseudoCommitTxnEvents(commitTxnEvent);
|
||||
@@ -230,14 +325,16 @@ public class EventExecutorService {
|
||||
pseudoCommitTxnEvent.getDbName());
|
||||
dbEventExecutor.enqueue(pseudoCommitTxnEvent);
|
||||
}
|
||||
return !pseudoCommitTxnEvents.isEmpty();
|
||||
}
|
||||
|
||||
/**
|
||||
* Dispatches pseudo abort transaction events to the tables involved in transaction for
|
||||
* processing.
|
||||
* @param abortTxnEvent Abort transaction event
|
||||
* @return True if any pseudo-events are dispatched. False otherwise.
|
||||
*/
|
||||
private void processAbortTxnEvent(AbortTxnEvent abortTxnEvent) {
|
||||
private boolean processAbortTxnEvent(AbortTxnEvent abortTxnEvent) {
|
||||
Set<TableWriteId> tableWriteIds = abortTxnEvent.getTableWriteIds();
|
||||
Map<TableName, List<Long>> tableWriteIdsMap = new HashMap<>();
|
||||
for (TableWriteId tableWriteId : tableWriteIds) {
|
||||
@@ -245,12 +342,14 @@ public class EventExecutorService {
|
||||
new TableName(tableWriteId.getDbName(), tableWriteId.getTblName()),
|
||||
k -> new ArrayList<>()).add(tableWriteId.getWriteId());
|
||||
}
|
||||
DerivedMetastoreEventContext context = new DerivedMetastoreEventContext(abortTxnEvent,
|
||||
tableWriteIdsMap.size());
|
||||
for (Map.Entry<TableName, List<Long>> entry : tableWriteIdsMap.entrySet()) {
|
||||
DbEventExecutor dbEventExecutor = getOrFindDbEventExecutor(entry.getKey().getDb());
|
||||
dbEventExecutor.enqueue(
|
||||
new PseudoAbortTxnEvent(abortTxnEvent, entry.getKey().getDb(),
|
||||
dbEventExecutor.enqueue(new PseudoAbortTxnEvent(context, entry.getKey().getDb(),
|
||||
entry.getKey().getTbl(), entry.getValue()));
|
||||
}
|
||||
return !tableWriteIdsMap.isEmpty();
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -287,8 +386,10 @@ public class EventExecutorService {
|
||||
|
||||
RenameTableBarrierEvent.RenameEventState state =
|
||||
new RenameTableBarrierEvent.RenameEventState();
|
||||
barrierEvents.add(new RenameTableBarrierEvent(alterEvent, dropTableEvent, state));
|
||||
barrierEvents.add(new RenameTableBarrierEvent(alterEvent, createTableEvent, state));
|
||||
DerivedMetastoreEventContext context = new DerivedMetastoreEventContext(alterEvent,
|
||||
2);
|
||||
barrierEvents.add(new RenameTableBarrierEvent(context, dropTableEvent, state));
|
||||
barrierEvents.add(new RenameTableBarrierEvent(context, createTableEvent, state));
|
||||
return barrierEvents;
|
||||
}
|
||||
|
||||
@@ -350,4 +451,131 @@ public class EventExecutorService {
|
||||
}
|
||||
return eventExecutor;
|
||||
}
|
||||
|
||||
/**
|
||||
* Records the given event as processed in processed log.
|
||||
* @param eventId Metastore event id
|
||||
* @param eventTime Metastore event time
|
||||
*/
|
||||
synchronized void addToProcessedLog(long eventId, long eventTime) {
|
||||
processedLog_.put(eventId, eventTime);
|
||||
if (processedLog_.size() > 1) pruneProcessedLog();
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds the given event to in-progress log.
|
||||
* <p>
|
||||
* Delimiter events are special markers used to preserve continuity and correctness
|
||||
* in synchronization tracking. They are not queued to DbEventExecutor, and are
|
||||
* automatically removed when their preceding non-delimiter event is removed.
|
||||
* @param event Metastore event to be tracked
|
||||
* @param isDelimiter True if the event is a delimiter event. False otherwise
|
||||
*/
|
||||
private synchronized void addToInProgressLog(MetastoreEvent event,
|
||||
boolean isDelimiter) {
|
||||
event.setDelimiter(isDelimiter);
|
||||
long currentTime = System.currentTimeMillis();
|
||||
// Log time taken to dispatch event. It is the time since the metastore event object
|
||||
// is created
|
||||
event.debugLog("Dispatch time: {}",
|
||||
PrintUtils.printTimeMs(currentTime - event.getCreationTime()));
|
||||
if (inProgressLog_.isEmpty() && isDelimiter) {
|
||||
// Event is considered implicitly processed and is directly recorded in the
|
||||
// processed log
|
||||
addToProcessedLog(event.getEventId(), event.getEvent().getEventTime());
|
||||
return;
|
||||
}
|
||||
// Initialize the event dispatch time. It is used to calculate the time taken to
|
||||
// process event, when event is removed from inProgressLog_
|
||||
event.setDispatchTime(currentTime);
|
||||
inProgressLog_.put(event.getEventId(), event);
|
||||
}
|
||||
|
||||
/**
|
||||
* Removes the specified event from the in-progress log and records it as processed.
|
||||
* <p>
|
||||
* This method also removes and marks any subsequent delimiter events as processed
|
||||
* until a non-delimiter event is encountered. This ensures the in-progress log
|
||||
* accurately reflects only unprocessed events.
|
||||
* <p>
|
||||
* After recording processed events, {@link #pruneProcessedLog()} method is called to
|
||||
* discard obsolete entries from the processed log.
|
||||
* @param eventId Id of the event to remove and mark as processed
|
||||
*/
|
||||
synchronized void removeFromInProgressLog(long eventId) {
|
||||
long currentTime = System.currentTimeMillis();
|
||||
MetastoreEvent event = inProgressLog_.remove(eventId);
|
||||
// event may be null when catalog is being reset. Metastore event processing and
|
||||
// catalog reset happens in different threads. In that case, clearLogs() would
|
||||
// have already cleared both inProgressLog_ and processedLog_.
|
||||
if (event == null) return;
|
||||
// Log time taken to process event. It is the time since the metastore event is
|
||||
// dispatched for processing
|
||||
event.debugLog("Complete process time: {}",
|
||||
PrintUtils.printTimeMs(currentTime - event.getDispatchTime()));
|
||||
processedLog_.put(eventId, (long) event.getEvent().getEventTime());
|
||||
Preconditions.checkState(!event.isDelimiter());
|
||||
// Remove all the subsequent delimiter events until a non-delimiter event is
|
||||
// encountered
|
||||
Iterator<Map.Entry<Long, MetastoreEvent>> it =
|
||||
inProgressLog_.tailMap(eventId).entrySet().iterator();
|
||||
while (it.hasNext()) {
|
||||
Map.Entry<Long, MetastoreEvent> entry = it.next();
|
||||
event = entry.getValue();
|
||||
if (!event.isDelimiter()) break;
|
||||
processedLog_.put(entry.getKey(), (long) event.getEvent().getEventTime());
|
||||
it.remove();
|
||||
}
|
||||
LOG.debug("Current count of dispatched events that are being tracked: {} ",
|
||||
inProgressLog_.size());
|
||||
pruneProcessedLog();
|
||||
}
|
||||
|
||||
/**
|
||||
* Prunes the processed event log by removing entries that are no longer needed for
|
||||
* synchronization tracking.
|
||||
* <p>
|
||||
* This method determines the greatest event id up to which all previous events are
|
||||
* guaranteed to have been synced and discards entries older than that event. The first
|
||||
* entry in the processed log is always considered the greatest synced event id.
|
||||
*/
|
||||
private synchronized void pruneProcessedLog() {
|
||||
long newGreatestSyncedEventId = processedLog_.lastKey();
|
||||
if (!inProgressLog_.isEmpty()) {
|
||||
Long firstUnprocessedEventId = inProgressLog_.firstKey();
|
||||
// First unprocessed event id is always greater than greatest synced event id
|
||||
Preconditions.checkState(firstUnprocessedEventId > processedLog_.firstKey());
|
||||
// Adjust the greatest synced event id to highest processed event id just before
|
||||
// first unprocessed event id.
|
||||
newGreatestSyncedEventId = processedLog_.lowerKey(firstUnprocessedEventId);
|
||||
}
|
||||
// Retain only the entries from the updated greatest synced event id onward
|
||||
processedLog_ = new TreeMap<>(processedLog_.tailMap(newGreatestSyncedEventId));
|
||||
LOG.debug("Current count of processed events that are tracked: {}, " +
|
||||
"greatest synced event id: {}", processedLog_.size(), newGreatestSyncedEventId);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the greatest synced event id. Greatest synced event is the latest event such
|
||||
* that all events with id less than or equal to the latest event are definitely synced.
|
||||
* @return Event id of the greatest synced event, or -1 if unknown.
|
||||
*/
|
||||
synchronized long getGreatestSyncedEventId() {
|
||||
long greatestSyncedEventId = -1;
|
||||
Map.Entry<Long, Long> entry = processedLog_.firstEntry();
|
||||
if (entry != null) greatestSyncedEventId = entry.getKey();
|
||||
return greatestSyncedEventId;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the greatest synced event time. Greatest synced event is the latest event such
|
||||
* that all events with id less than or equal to the latest event are definitely synced.
|
||||
* @return Time of the greatest synced event, or -1 if unknown.
|
||||
*/
|
||||
synchronized long getGreatestSyncedEventTime() {
|
||||
long greatestSyncedEventTime = 0;
|
||||
Map.Entry<Long, Long> entry = processedLog_.firstEntry();
|
||||
if (entry != null) greatestSyncedEventTime = entry.getValue();
|
||||
return greatestSyncedEventTime;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,49 @@
|
||||
// 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.common.ImpalaException;
|
||||
|
||||
|
||||
/**
|
||||
* Wrapper exception class thrown for errors during event processing in
|
||||
* {@link DbEventExecutor.DbProcessor#process()} and
|
||||
* {@link TableEventExecutor.TableProcessor#process()}
|
||||
*/
|
||||
public class EventProcessException extends ImpalaException {
|
||||
// Notification event under processing when exception occurred
|
||||
private final NotificationEvent event_;
|
||||
|
||||
// Inherent exception during event process
|
||||
private final Exception exception_;
|
||||
|
||||
public EventProcessException(NotificationEvent event, Exception exception) {
|
||||
super(exception);
|
||||
event_ = event;
|
||||
exception_ = exception;
|
||||
}
|
||||
|
||||
public Exception getException() {
|
||||
return exception_;
|
||||
}
|
||||
|
||||
public NotificationEvent getEvent() {
|
||||
return event_;
|
||||
}
|
||||
}
|
||||
@@ -52,6 +52,12 @@ public interface ExternalEventsProcessor {
|
||||
*/
|
||||
void pause();
|
||||
|
||||
/**
|
||||
* Gracefully Pauses the event processing. Use <code>start(fromEventId)</code> method
|
||||
* below to restart the event processing
|
||||
*/
|
||||
void pauseGracefully();
|
||||
|
||||
/**
|
||||
* Starts the event processing from the given eventId. This method can be used to jump
|
||||
* ahead in the event processing under certain cases where it is okay skip certain
|
||||
|
||||
@@ -34,6 +34,7 @@ import java.util.Objects;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.TreeMap;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
@@ -629,6 +630,23 @@ public class MetastoreEvents {
|
||||
// metrics registry so that events can add metrics
|
||||
protected final Metrics metrics_;
|
||||
|
||||
// Event creation time. It is initialized in constructor. It is not used for
|
||||
// DerivedMetastoreEvent.
|
||||
protected final long creationTime_;
|
||||
|
||||
// Event dispatch time. It is initialized after dispatching actual event. It is not
|
||||
// set for DerivedMetastoreEvent.
|
||||
protected long dispatchTime_;
|
||||
|
||||
// Whether it is delimiter event. It is set only when hierarchical event processing is
|
||||
// enabled. Delimiter is a kind of metastore event that do not require event
|
||||
// processing. Delimeter event can be:
|
||||
// 1. A CommitTxnEvent that do not have any write event info for a given transaction.
|
||||
// 2. An AbortTxnEvent that do not have write ids for a given transaction.
|
||||
// 3. An IgnoredEvent.
|
||||
// An event is determined and marked as delimiter in EventExecutorService#dispatch()
|
||||
protected boolean isDelimiter_;
|
||||
|
||||
protected MetastoreEvent(CatalogOpExecutor catalogOpExecutor, Metrics metrics,
|
||||
NotificationEvent event) {
|
||||
this.catalogOpExecutor_ = catalogOpExecutor;
|
||||
@@ -642,6 +660,54 @@ public class MetastoreEvents {
|
||||
this.tblName_ = event.getTableName();
|
||||
this.metastoreNotificationEvent_ = event;
|
||||
this.metrics_ = metrics;
|
||||
this.creationTime_ = System.currentTimeMillis();
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets metastore event creation time.
|
||||
* @return Creation time
|
||||
*/
|
||||
long getCreationTime() {
|
||||
return creationTime_;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets metastore event dispatch time.
|
||||
* @return Dispatch time
|
||||
*/
|
||||
long getDispatchTime() {
|
||||
if (this instanceof DerivedMetastoreEvent) {
|
||||
return ((DerivedMetastoreEvent) this).getActualEvent().dispatchTime_;
|
||||
}
|
||||
return dispatchTime_;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the metastore event dispatch time. It can be initialized only once after
|
||||
* dispatching the event.
|
||||
* @param dispatchTime Time when event is dispatched
|
||||
*/
|
||||
void setDispatchTime(long dispatchTime) {
|
||||
Preconditions.checkState(dispatchTime_ == 0, "Dispatch time can be set only once");
|
||||
Preconditions.checkState(!(this instanceof DerivedMetastoreEvent),
|
||||
"Not allowed to set on derived metastore event currently");
|
||||
dispatchTime_ = dispatchTime;
|
||||
}
|
||||
|
||||
/**
|
||||
* To determine whether it is a delimiter event.
|
||||
* @return True if delimiter event. False otherwise
|
||||
*/
|
||||
public boolean isDelimiter() {
|
||||
return isDelimiter_;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the event as delimiter.
|
||||
* @param delimiter
|
||||
*/
|
||||
public void setDelimiter(boolean delimiter) {
|
||||
isDelimiter_ = delimiter;
|
||||
}
|
||||
|
||||
public long getEventId() { return eventId_; }
|
||||
@@ -714,16 +780,21 @@ public class MetastoreEvents {
|
||||
*/
|
||||
public void processIfEnabled()
|
||||
throws CatalogException, MetastoreNotificationException {
|
||||
boolean isNotDerivedEvent = !(this instanceof DerivedMetastoreEvent);
|
||||
if (isEventProcessingDisabled()) {
|
||||
infoLog("Skipping this event because of flag evaluation");
|
||||
if (isNotDerivedEvent) {
|
||||
metrics_.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).inc();
|
||||
debugLog("Incremented skipped metric to " + metrics_
|
||||
.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).getCount());
|
||||
debugLog("Incremented skipped metric to " + metrics_.getCounter(
|
||||
MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).getCount());
|
||||
}
|
||||
return;
|
||||
}
|
||||
if (BackendConfig.INSTANCE.enableSyncToLatestEventOnDdls()) {
|
||||
if (shouldSkipWhenSyncingToLatestEventId()) {
|
||||
if (isNotDerivedEvent) {
|
||||
metrics_.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).inc();
|
||||
}
|
||||
return;
|
||||
}
|
||||
}
|
||||
@@ -996,9 +1067,124 @@ public class MetastoreEvents {
|
||||
}
|
||||
|
||||
/**
|
||||
* Marker interface for derived metastore event
|
||||
* Interface for derived metastore event
|
||||
*/
|
||||
public interface DerivedMetastoreEvent {
|
||||
// Gets the actual metastore event.
|
||||
MetastoreEvent getActualEvent();
|
||||
|
||||
// Marks the derived event as processed.
|
||||
void markProcessed();
|
||||
|
||||
// Determines whether all the derived event associated with actual event are
|
||||
// processed.
|
||||
boolean isAllDerivedEventsProcessed();
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
public static class DerivedMetastoreEventContext {
|
||||
// Total count of derived table events associated to actual table event. When a
|
||||
// derived event is processed, this count is decremented by 1. Once the count becomes
|
||||
// 0, actual table event is treated as processed.
|
||||
private final AtomicInteger derivedEventsCount_;
|
||||
|
||||
// Actual metastore event the event is derived from.
|
||||
private final MetastoreEvent actualEvent_;
|
||||
|
||||
public DerivedMetastoreEventContext(MetastoreEvent actualEvent,
|
||||
int derivedEventsCount) {
|
||||
Preconditions.checkArgument(actualEvent != null);
|
||||
derivedEventsCount_ = new AtomicInteger(derivedEventsCount);
|
||||
actualEvent_ = actualEvent;
|
||||
}
|
||||
|
||||
public DerivedMetastoreEventContext(MetastoreEvent actualEvent) {
|
||||
Preconditions.checkArgument(actualEvent != null);
|
||||
derivedEventsCount_ = new AtomicInteger(0);
|
||||
actualEvent_ = actualEvent;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the derived metastore event count. Need to be initialized only once.
|
||||
* @param count Derived metastore event count
|
||||
*/
|
||||
public void setDerivedEventsCount(int count) {
|
||||
Preconditions.checkState(derivedEventsCount_.get() == 0,
|
||||
"Derived events count can be set only once");
|
||||
derivedEventsCount_.set(count);
|
||||
}
|
||||
|
||||
public MetastoreEvent getActualEvent() {
|
||||
return actualEvent_;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Abstract class for derived metastore table event. A single actual MetastoreEvent can
|
||||
* have multiple DerivedMetastoreTableEvent associated with it.
|
||||
*/
|
||||
public static abstract class DerivedMetastoreTableEvent extends MetastoreTableEvent
|
||||
implements DerivedMetastoreEvent {
|
||||
|
||||
private final DerivedMetastoreEventContext context_;
|
||||
|
||||
// Whether the current derived table event is processed.
|
||||
private boolean isProcessed_ = false;
|
||||
|
||||
DerivedMetastoreTableEvent(CatalogOpExecutor catalogOpExecutor, Metrics metrics,
|
||||
NotificationEvent event, DerivedMetastoreEventContext context) {
|
||||
super(catalogOpExecutor, metrics, event);
|
||||
Preconditions.checkArgument(context.derivedEventsCount_ != null);
|
||||
context_ = context;
|
||||
}
|
||||
|
||||
DerivedMetastoreTableEvent(DerivedMetastoreEventContext context,
|
||||
NotificationEvent event) {
|
||||
this(context.getActualEvent().getCatalogOpExecutor(),
|
||||
context.getActualEvent().getMetrics(), event, context);
|
||||
}
|
||||
|
||||
protected DerivedMetastoreTableEvent(DerivedMetastoreEventContext context) {
|
||||
this(context.getActualEvent().getCatalogOpExecutor(),
|
||||
context.getActualEvent().getMetrics(), context.getActualEvent().getEvent(),
|
||||
context);
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the actual metastore event.
|
||||
* @return Actual metastore event this event is associated to
|
||||
*/
|
||||
@Override
|
||||
public MetastoreEvent getActualEvent() {
|
||||
return context_.getActualEvent();
|
||||
}
|
||||
|
||||
/**
|
||||
* Marks the current derived event as processed.
|
||||
* <p>
|
||||
* This method must be called only once per derived event instance. It decrements the
|
||||
* count of remaining derived events associated with the actual event.
|
||||
*/
|
||||
@Override
|
||||
public void markProcessed() {
|
||||
Preconditions.checkState(!isProcessed_, "Event already processed");
|
||||
isProcessed_ = true;
|
||||
Preconditions.checkState(context_.derivedEventsCount_.decrementAndGet() >= 0);
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines whether the all the derived event associated with actual event are
|
||||
* processed. The actual event is treated as processed only when all its associated
|
||||
* derived events are processed.
|
||||
*
|
||||
* @return True if all derived events are processed. False otherwise.
|
||||
*/
|
||||
@Override
|
||||
public boolean isAllDerivedEventsProcessed() {
|
||||
return context_.derivedEventsCount_.get() == 0;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -3491,15 +3677,14 @@ public class MetastoreEvents {
|
||||
/**
|
||||
* Pseudo abort transaction event handles abort processing for a table
|
||||
*/
|
||||
public static class PseudoAbortTxnEvent extends MetastoreTableEvent
|
||||
implements DerivedMetastoreEvent {
|
||||
public static class PseudoAbortTxnEvent extends DerivedMetastoreTableEvent {
|
||||
final private long txnId_;
|
||||
final private List<Long> writeIds_;
|
||||
|
||||
PseudoAbortTxnEvent(AbortTxnEvent actualEvent, String dbName, String tableName,
|
||||
List<Long> writeIds) {
|
||||
super(actualEvent.catalogOpExecutor_, actualEvent.metrics_, actualEvent.event_);
|
||||
txnId_ = actualEvent.txnId_;
|
||||
PseudoAbortTxnEvent(DerivedMetastoreEventContext context, String dbName,
|
||||
String tableName, List<Long> writeIds) {
|
||||
super(context);
|
||||
txnId_ = ((AbortTxnEvent) context.getActualEvent()).txnId_;
|
||||
writeIds_ = writeIds;
|
||||
dbName_ = dbName;
|
||||
tblName_ = tableName;
|
||||
@@ -3526,8 +3711,15 @@ public class MetastoreEvents {
|
||||
|
||||
@Override
|
||||
protected boolean isEventProcessingDisabled() {
|
||||
org.apache.impala.catalog.Table tbl = catalog_.getTableNoThrow(dbName_, tblName_);
|
||||
if (tbl != null && tbl.getCreateEventId() < getEventId()) {
|
||||
msTbl_ = tbl.getMetaStoreTable();
|
||||
}
|
||||
if (msTbl_ == null) {
|
||||
return false;
|
||||
}
|
||||
return super.isEventProcessingDisabled();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected SelfEventContext getSelfEventContext() {
|
||||
|
||||
@@ -276,6 +276,10 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
|
||||
public static final String LAST_SYNCED_ID_METRIC = "last-synced-event-id";
|
||||
// last synced event time
|
||||
public static final String LAST_SYNCED_EVENT_TIME = "last-synced-event-time";
|
||||
// Greatest synced event id
|
||||
public static final String GREATEST_SYNCED_EVENT_ID = "greatest-synced-event-id";
|
||||
// Greatest synced event time
|
||||
public static final String GREATEST_SYNCED_EVENT_TIME = "greatest-synced-event-time";
|
||||
// latest event id in Hive metastore
|
||||
public static final String LATEST_EVENT_ID = "latest-event-id";
|
||||
// event time of the latest event in Hive metastore
|
||||
@@ -802,10 +806,14 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
|
||||
metrics_.addGauge(LAST_SYNCED_ID_METRIC, (Gauge<Long>) lastSyncedEventId_::get);
|
||||
metrics_.addGauge(LAST_SYNCED_EVENT_TIME,
|
||||
(Gauge<Long>) lastSyncedEventTimeSecs_::get);
|
||||
metrics_.addGauge(GREATEST_SYNCED_EVENT_ID,
|
||||
(Gauge<Long>) this::getGreatestSyncedEventId);
|
||||
metrics_.addGauge(GREATEST_SYNCED_EVENT_TIME,
|
||||
(Gauge<Long>) this::getGreatestSyncedEventTime);
|
||||
metrics_.addGauge(LATEST_EVENT_ID, (Gauge<Long>) latestEventId_::get);
|
||||
metrics_.addGauge(LATEST_EVENT_TIME, (Gauge<Long>) latestEventTimeSecs_::get);
|
||||
metrics_.addGauge(EVENT_PROCESSING_DELAY,
|
||||
(Gauge<Long>) () -> latestEventTimeSecs_.get() - lastSyncedEventTimeSecs_.get());
|
||||
(Gauge<Long>) () -> latestEventTimeSecs_.get() - getGreatestSyncedEventTime());
|
||||
metrics_.addCounter(NUMBER_OF_TABLE_REFRESHES);
|
||||
metrics_.addCounter(NUMBER_OF_PARTITION_REFRESHES);
|
||||
metrics_.addCounter(NUMBER_OF_TABLES_ADDED);
|
||||
@@ -869,6 +877,15 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
|
||||
return eventProcessorStatus_;
|
||||
}
|
||||
|
||||
/**
|
||||
* Whether metastore event can be processed in current status.
|
||||
* @return True if event can be processed. False Otherwise.
|
||||
*/
|
||||
public boolean canProcessEventInCurrentStatus() {
|
||||
return (eventProcessorStatus_ == EventProcessorStatus.ACTIVE) ||
|
||||
(eventProcessorStatus_ == EventProcessorStatus.PAUSED);
|
||||
}
|
||||
|
||||
/**
|
||||
* This method is only for testing.
|
||||
* @return EventExecutorService
|
||||
@@ -933,6 +950,58 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
|
||||
return lastSyncedEventId_.get();
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the current value of lastSyncedEventTimeSecs.
|
||||
* @return
|
||||
*/
|
||||
public long getLastSyncedEventTime() {
|
||||
return lastSyncedEventTimeSecs_.get();
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the greatest synced event id. Greatest synced event is the latest event such
|
||||
* that all events with id less than or equal to the latest event are definitely synced.
|
||||
* When hierarchical event processing is not enabled, it is the same as the last synced
|
||||
* event id.
|
||||
* @return Event id of the greatest synced event
|
||||
*/
|
||||
public long getGreatestSyncedEventId() {
|
||||
if (isHierarchicalEventProcessingEnabled()) {
|
||||
return eventExecutorService_.getGreatestSyncedEventId();
|
||||
}
|
||||
return getLastSyncedEventId();
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the greatest synced event time. Greatest synced event is the latest event such
|
||||
* that all events with id less than or equal to the latest event are definitely synced.
|
||||
* When hierarchical event processing is not enabled, it is the same as the last synced
|
||||
* event time.
|
||||
* @return Time of the greatest synced event
|
||||
*/
|
||||
public long getGreatestSyncedEventTime() {
|
||||
if (isHierarchicalEventProcessingEnabled()) {
|
||||
return eventExecutorService_.getGreatestSyncedEventTime();
|
||||
}
|
||||
return getLastSyncedEventTime();
|
||||
}
|
||||
|
||||
/**
|
||||
* Calculates the number of metastore events that are pending synchronization up to the
|
||||
* specified {@code latestEventId}.
|
||||
* @param latestEventId Latest event id available on HMS
|
||||
* @return Number of events that need to be synchronized. Returns 0 if all events up to
|
||||
* {@code latestEventId} have already been synchronized.
|
||||
*/
|
||||
public long getPendingEventCount(long latestEventId) {
|
||||
if (isHierarchicalEventProcessingEnabled()) {
|
||||
return eventExecutorService_.getPendingEventCount(latestEventId);
|
||||
}
|
||||
long lastSyncedEventId = lastSyncedEventId_.get();
|
||||
if (latestEventId <= lastSyncedEventId) return 0;
|
||||
return latestEventId - lastSyncedEventId;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the current value of latestEventId_. This method is not thread-safe and
|
||||
* only to be used for testing purposes
|
||||
@@ -970,8 +1039,64 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
|
||||
return;
|
||||
}
|
||||
updateStatus(EventProcessorStatus.PAUSED);
|
||||
LOG.info(String.format("Event processing is paused. Last synced event id is %d",
|
||||
lastSyncedEventId_.get()));
|
||||
LOG.info("Event processing is paused. {} synced event id is {}",
|
||||
isHierarchicalEventProcessingEnabled() ? "Greatest" : "Last",
|
||||
getGreatestSyncedEventId());
|
||||
}
|
||||
|
||||
/**
|
||||
* Gracefully pauses the event processor by setting its status to
|
||||
* <code>EventProcessorStatus.PAUSED</code>.
|
||||
* <p>
|
||||
* Ensures that all currently fetched events from HMS are processed before the processor
|
||||
* is fully paused.
|
||||
* No new events will be fetched for processing while the processor remains in the
|
||||
* paused state.
|
||||
*/
|
||||
@Override
|
||||
public void pauseGracefully() {
|
||||
pause();
|
||||
LOG.info("Process already fetched events if any");
|
||||
if (isHierarchicalEventProcessingEnabled()) {
|
||||
ensureEventsProcessedInHierarchicalMode(3600000);
|
||||
} else {
|
||||
boolean isNeedWait = isCurrentFilteredEventsExist();
|
||||
while (isNeedWait) {
|
||||
Uninterruptibles.sleepUninterruptibly(5, TimeUnit.MILLISECONDS);
|
||||
isNeedWait = isCurrentFilteredEventsExist();
|
||||
}
|
||||
}
|
||||
LOG.info("Event processing is paused. Greatest synced event id is {}",
|
||||
getGreatestSyncedEventId());
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines whether current filtered metastore events exists.
|
||||
* @return
|
||||
*/
|
||||
public synchronized boolean isCurrentFilteredEventsExist() {
|
||||
return currentFilteredEvents_ != null && !currentFilteredEvents_.isEmpty();
|
||||
}
|
||||
|
||||
/**
|
||||
* Resets the current filtered metastore events.
|
||||
*/
|
||||
public synchronized void resetCurrentFilteredEvents() {
|
||||
currentFilteredEvents_ = null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Populates the current filtered metastore events from the given HMS notification
|
||||
* events.
|
||||
* @param events List of NotificationEvent
|
||||
* @throws MetastoreNotificationException
|
||||
*/
|
||||
public synchronized void populateCurrentFilteredEvents(List<NotificationEvent> events)
|
||||
throws MetastoreNotificationException {
|
||||
currentFilteredEvents_ = Collections.emptyList();
|
||||
// Do not create metastore events for this batch of HMS events if status is not active
|
||||
if (eventProcessorStatus_ != EventProcessorStatus.ACTIVE) return;
|
||||
currentFilteredEvents_ = metastoreEventFactory_.getFilteredEvents(events, metrics_);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -1043,7 +1168,7 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
|
||||
public synchronized void start(long fromEventId) {
|
||||
Preconditions.checkArgument(fromEventId >= 0);
|
||||
EventProcessorStatus currentStatus = eventProcessorStatus_;
|
||||
long prevLastSyncedEventId = lastSyncedEventId_.get();
|
||||
long prevLastSyncedEventId = getGreatestSyncedEventId();
|
||||
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
|
||||
@@ -1056,8 +1181,11 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
|
||||
// Clear the error message of the last failure and reset the progress info
|
||||
eventProcessorErrorMsg_ = null;
|
||||
resetProgressInfo();
|
||||
// Clear delete event log
|
||||
deleteEventLog_.garbageCollect(fromEventId);
|
||||
lastSyncedEventId_.set(fromEventId);
|
||||
lastSyncedEventTimeSecs_.set(getEventTimeFromHMS(fromEventId));
|
||||
if (isHierarchicalEventProcessingEnabled()) eventExecutorService_.start();
|
||||
updateStatus(EventProcessorStatus.ACTIVE);
|
||||
LOG.info(String.format(
|
||||
"Metastore event processing restarted. Last synced event id was updated "
|
||||
@@ -1229,9 +1357,11 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
|
||||
List<NotificationEvent> events = getNextMetastoreEvents(currentEventId);
|
||||
processEvents(currentEventId, events);
|
||||
} catch (Exception ex) {
|
||||
handleEventProcessException(ex);
|
||||
handleEventProcessException(ex, currentEvent_);
|
||||
} finally {
|
||||
if (isHierarchicalEventProcessingEnabled()) {
|
||||
// Do maintenance cleanup to remove idle DbProcessors if any, regardless of
|
||||
// exception occurrence or not.
|
||||
eventExecutorService_.cleanup();
|
||||
}
|
||||
}
|
||||
@@ -1246,8 +1376,9 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
|
||||
* MetastoreNotificationFetchException, since event processor and event executor
|
||||
* service will continue to process events once the HMS is UP again.
|
||||
* @param ex Exception in event processing
|
||||
* @param event HMS Notification event when the exception occurred
|
||||
*/
|
||||
public void handleEventProcessException(Exception ex) {
|
||||
public void handleEventProcessException(Exception ex, NotificationEvent event) {
|
||||
if (ex instanceof MetastoreNotificationFetchException) {
|
||||
// No need to change the EventProcessor state to error since we want the
|
||||
// EventProcessor to continue getting new events after HMS is back up.
|
||||
@@ -1267,9 +1398,8 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
|
||||
updateStatus(EventProcessorStatus.ERROR);
|
||||
String msg = "Unexpected exception received while processing event";
|
||||
LOG.error(msg, ex);
|
||||
eventProcessorErrorMsg_ = LocalDateTime.now().toString() + '\n' + msg + '\n' +
|
||||
ExceptionUtils.getStackTrace(ex);
|
||||
dumpEventInfoToLog(currentEvent_);
|
||||
dumpEventInfoToLog(event, LocalDateTime.now().toString() + '\n' + msg +
|
||||
'\n' + ExceptionUtils.getStackTrace(ex));
|
||||
tryAutoGlobalInvalidateOnFailure();
|
||||
}
|
||||
}
|
||||
@@ -1316,25 +1446,19 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
|
||||
NotificationEvent event = getEventFromHMS(msClient, currentEventId);
|
||||
// Events could be empty if they are just cleaned up.
|
||||
if (event == null) return;
|
||||
long lastSyncedEventId = lastSyncedEventId_.get();
|
||||
long lastSyncedEventTime = lastSyncedEventTimeSecs_.get();
|
||||
long lastSyncedEventId = getGreatestSyncedEventId();
|
||||
long lastSyncedEventTime = getGreatestSyncedEventTime();
|
||||
long currentEventTime = event.getEventTime();
|
||||
latestEventId_.set(currentEventId);
|
||||
latestEventTimeSecs_.set(currentEventTime);
|
||||
LOG.info("Latest event in HMS: id={}, time={}. Last synced event: id={}, time={}.",
|
||||
currentEventId, currentEventTime, lastSyncedEventId, lastSyncedEventTime);
|
||||
LOG.info("Latest event in HMS: id={}, time={}. {} synced event: id={}, time={}.",
|
||||
currentEventId, currentEventTime,
|
||||
isHierarchicalEventProcessingEnabled() ? "Greatest" : "Last",
|
||||
lastSyncedEventId, lastSyncedEventTime);
|
||||
if (currentEventTime > lastSyncedEventTime) {
|
||||
// TODO: Need to redefine the lag in the hierarchical processing mode.
|
||||
// Hierarchical mode currently have a mechanism to check the total number of
|
||||
// outstanding events at the moment so that memory usage pressure on catalogd
|
||||
// can be controlled when event processing becomes slow i.e., iff outstanding
|
||||
// event count exceeds max_outstanding_events_on_executors configured value. And
|
||||
// lastSyncedEventTimeSecs_ accounts only for event dispatch time in hierarchical
|
||||
// mode.
|
||||
LOG.warn("Lag: {}. {} events pending to be {}.",
|
||||
LOG.warn("Lag: {}. Approximately {} events pending to be processed.",
|
||||
PrintUtils.printTimeMs((currentEventTime - lastSyncedEventTime) * 1000),
|
||||
currentEventId - lastSyncedEventId,
|
||||
isHierarchicalEventProcessingEnabled() ? "dispatched" : "processed");
|
||||
getPendingEventCount(currentEventId));
|
||||
}
|
||||
} catch (Exception e) {
|
||||
LOG.error("Unable to update current notification event id. Last value: {}",
|
||||
@@ -1351,10 +1475,14 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
|
||||
TEventProcessorMetrics eventProcessorMetrics = new TEventProcessorMetrics();
|
||||
EventProcessorStatus currentStatus = getStatus();
|
||||
eventProcessorMetrics.setStatus(currentStatus.toString());
|
||||
eventProcessorMetrics.setLast_synced_event_id(getLastSyncedEventId());
|
||||
eventProcessorMetrics.setLast_synced_event_id(getGreatestSyncedEventId());
|
||||
eventProcessorMetrics.setGreatest_synced_event_id(getGreatestSyncedEventId());
|
||||
if (currentStatus != EventProcessorStatus.ACTIVE) return eventProcessorMetrics;
|
||||
// The following counters are only updated when event-processor is active.
|
||||
eventProcessorMetrics.setLast_synced_event_time(lastSyncedEventTimeSecs_.get());
|
||||
eventProcessorMetrics.setLast_synced_event_time(getGreatestSyncedEventTime());
|
||||
eventProcessorMetrics.setGreatest_synced_event_time(getGreatestSyncedEventTime());
|
||||
eventProcessorMetrics.setPending_event_count(
|
||||
getPendingEventCount(latestEventId_.get()));
|
||||
eventProcessorMetrics.setLatest_event_id(latestEventId_.get());
|
||||
eventProcessorMetrics.setLatest_event_time(latestEventTimeSecs_.get());
|
||||
|
||||
@@ -1417,8 +1545,8 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
|
||||
summaryResponse.setError_msg(eventProcessorErrorMsg_);
|
||||
}
|
||||
TEventBatchProgressInfo progressInfo = new TEventBatchProgressInfo();
|
||||
progressInfo.last_synced_event_id = lastSyncedEventId_.get();
|
||||
progressInfo.last_synced_event_time_s = lastSyncedEventTimeSecs_.get();
|
||||
progressInfo.last_synced_event_id = getGreatestSyncedEventId();
|
||||
progressInfo.last_synced_event_time_s = getGreatestSyncedEventTime();
|
||||
progressInfo.latest_event_id = latestEventId_.get();
|
||||
progressInfo.latest_event_time_s = latestEventTimeSecs_.get();
|
||||
if (req.get_latest_event_from_hms) {
|
||||
@@ -1468,7 +1596,7 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
|
||||
currentEvent_ = null;
|
||||
currentEventBatch_ = null;
|
||||
currentFilteredEvent_ = null;
|
||||
currentFilteredEvents_ = null;
|
||||
resetCurrentFilteredEvents();
|
||||
currentBatchStartTimeMs_ = 0;
|
||||
currentEventStartTimeMs_ = 0;
|
||||
currentEventIndex_ = 0;
|
||||
@@ -1485,14 +1613,19 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
|
||||
protected void processEvents(long currentEventId, List<NotificationEvent> events)
|
||||
throws MetastoreNotificationException {
|
||||
currentEventBatch_ = events;
|
||||
boolean isHierarchical = isHierarchicalEventProcessingEnabled();
|
||||
// update the events received metric before returning
|
||||
metrics_.getMeter(EVENTS_RECEIVED_METRIC).mark(events.size());
|
||||
if (events.isEmpty()) {
|
||||
if (lastSyncedEventId_.get() < currentEventId) {
|
||||
// Possible to receive empty list due to event skip list in notification event
|
||||
// request. Update the last synced event id with current event id on metastore
|
||||
long currentEventTime = getEventTimeFromHMS(currentEventId);
|
||||
lastSyncedEventId_.set(currentEventId);
|
||||
lastSyncedEventTimeSecs_.set(getEventTimeFromHMS(currentEventId));
|
||||
lastSyncedEventTimeSecs_.set(currentEventTime);
|
||||
if (isHierarchical) {
|
||||
eventExecutorService_.addToProcessedLog(currentEventId, currentEventTime);
|
||||
}
|
||||
}
|
||||
return;
|
||||
}
|
||||
@@ -1501,22 +1634,23 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
|
||||
currentBatchStartTimeMs_ = System.currentTimeMillis();
|
||||
Map<MetastoreEvent, Long> eventProcessingTime = new HashMap<>();
|
||||
try {
|
||||
currentFilteredEvents_ =
|
||||
metastoreEventFactory_.getFilteredEvents(events, metrics_);
|
||||
populateCurrentFilteredEvents(events);
|
||||
if (currentFilteredEvents_.isEmpty()) {
|
||||
NotificationEvent e = events.get(events.size() - 1);
|
||||
lastSyncedEventId_.set(e.getEventId());
|
||||
lastSyncedEventTimeSecs_.set(e.getEventTime());
|
||||
if (isHierarchical) {
|
||||
eventExecutorService_.addToProcessedLog(e.getEventId(), e.getEventTime());
|
||||
}
|
||||
resetProgressInfo();
|
||||
return;
|
||||
}
|
||||
boolean isHierarchical = isHierarchicalEventProcessingEnabled();
|
||||
for (MetastoreEvent event : currentFilteredEvents_) {
|
||||
// synchronizing each event processing reduces the scope of the lock so the a
|
||||
// potential reset() during event processing is not blocked for longer than
|
||||
// necessary
|
||||
synchronized (this) {
|
||||
if (eventProcessorStatus_ != EventProcessorStatus.ACTIVE) {
|
||||
if (!canProcessEventInCurrentStatus()) {
|
||||
break;
|
||||
}
|
||||
currentEvent_ = event.metastoreNotificationEvent_;
|
||||
@@ -1570,10 +1704,6 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
|
||||
|
||||
private void logEventMetrics(Map<MetastoreEvent, Long> eventProcessingTime,
|
||||
long elapsedNs) {
|
||||
// TODO: eventProcessingTime here is more relevant in sequential processing mode. But,
|
||||
// in case of hierarchical processing mode, these are the times taken at dispatcher
|
||||
// thread alone for each event. Need to have another mechanism to capture the actual
|
||||
// event processing time in hierarchical processing mode.
|
||||
boolean isHierarchical = isHierarchicalEventProcessingEnabled();
|
||||
LOG.info("Time elapsed in {} event batch: {}",
|
||||
isHierarchical ? "dispatching" : "processing",
|
||||
@@ -1623,11 +1753,11 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
|
||||
eventProcessorStatus_ = toStatus;
|
||||
}
|
||||
|
||||
private void dumpEventInfoToLog(NotificationEvent event) {
|
||||
private void dumpEventInfoToLog(NotificationEvent event, String errorMessage) {
|
||||
if (event == null) {
|
||||
String error = "Notification event is null";
|
||||
LOG.error(error);
|
||||
eventProcessorErrorMsg_ += '\n' + error;
|
||||
eventProcessorErrorMsg_ = errorMessage + '\n' + error;
|
||||
return;
|
||||
}
|
||||
StringBuilder msg =
|
||||
@@ -1641,7 +1771,7 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
|
||||
msg.append("Event message: ").append(event.getMessage()).append("\n");
|
||||
String msgStr = msg.toString();
|
||||
LOG.error(msgStr);
|
||||
eventProcessorErrorMsg_ += '\n' + msgStr;
|
||||
eventProcessorErrorMsg_ = errorMessage + '\n' + msgStr;
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -1760,8 +1890,7 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
|
||||
TStatus res = new TStatus();
|
||||
// Only waits when event-processor is in ACTIVE/PAUSED states. PAUSED states happen
|
||||
// at startup or when global invalidate is running, so it's ok to wait for.
|
||||
if (!EventProcessorStatus.ACTIVE.equals(eventProcessorStatus_)
|
||||
&& !EventProcessorStatus.PAUSED.equals(eventProcessorStatus_)) {
|
||||
if (!canProcessEventInCurrentStatus()) {
|
||||
res.setStatus_code(TErrorCode.GENERAL);
|
||||
res.addToError_msgs(
|
||||
"Current state of HMS event processor is " + eventProcessorStatus_);
|
||||
@@ -1784,7 +1913,7 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
|
||||
latestEventId, e);
|
||||
waitForEventId = latestEventId;
|
||||
}
|
||||
long lastSyncedEventId = getLastSyncedEventId();
|
||||
long lastSyncedEventId = getGreatestSyncedEventId();
|
||||
long startMs = System.currentTimeMillis();
|
||||
long sleepIntervalMs = Math.min(timeoutMs, hmsEventSyncSleepIntervalMs_);
|
||||
// Avoid too many log entries if the waiting interval is smaller than 500ms.
|
||||
@@ -1797,9 +1926,8 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
|
||||
lastSyncedEventId, waitForEventId);
|
||||
}
|
||||
Uninterruptibles.sleepUninterruptibly(sleepIntervalMs, TimeUnit.MILLISECONDS);
|
||||
lastSyncedEventId = getLastSyncedEventId();
|
||||
if (!EventProcessorStatus.ACTIVE.equals(eventProcessorStatus_)
|
||||
&& !EventProcessorStatus.PAUSED.equals(eventProcessorStatus_)) {
|
||||
lastSyncedEventId = getGreatestSyncedEventId();
|
||||
if (!canProcessEventInCurrentStatus()) {
|
||||
res.setStatus_code(TErrorCode.GENERAL);
|
||||
res.addToError_msgs(
|
||||
"Current state of HMS event processor is " + eventProcessorStatus_);
|
||||
@@ -1828,7 +1956,7 @@ public class MetastoreEventsProcessor implements ExternalEventsProcessor {
|
||||
// requiredEventId starts from the last synced event id. While checking pending
|
||||
// events of a target, we just fetch events after requiredEventId, since events
|
||||
// before it are decided to be waited for.
|
||||
long requiredEventId = getLastSyncedEventId();
|
||||
long requiredEventId = getGreatestSyncedEventId();
|
||||
if (req.want_db_list) {
|
||||
return getMinRequiredEventIdForDbList(requiredEventId);
|
||||
}
|
||||
|
||||
@@ -73,6 +73,11 @@ public class NoOpEventProcessor implements ExternalEventsProcessor {
|
||||
// no-op
|
||||
}
|
||||
|
||||
@Override
|
||||
public void pauseGracefully() {
|
||||
// no-op
|
||||
}
|
||||
|
||||
@Override
|
||||
public void start(long fromEventId) {
|
||||
// no-op
|
||||
|
||||
@@ -24,8 +24,8 @@ import com.google.common.base.Preconditions;
|
||||
import org.apache.impala.catalog.CatalogException;
|
||||
import org.apache.impala.catalog.Db;
|
||||
import org.apache.impala.catalog.Table;
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.AlterTableEvent;
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.DerivedMetastoreEvent;
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.DerivedMetastoreEventContext;
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.DerivedMetastoreTableEvent;
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEventType;
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.MetastoreTableEvent;
|
||||
import org.apache.impala.common.Reference;
|
||||
@@ -36,14 +36,15 @@ import static org.apache.impala.catalog.Table.TBL_EVENTS_PROCESS_DURATION;
|
||||
|
||||
/**
|
||||
* An instance of this class is used to synchronize the source and target TableProcessors
|
||||
* to process rename table event. It holds the reference to alter table event,
|
||||
* pseudo-event(either drop or create table) and the combined processing state of the
|
||||
* pseudo-events.
|
||||
* to process rename table event. It holds the reference to alter table event through
|
||||
* context, pseudo-event(either drop or create table) and the combined processing state
|
||||
* of the pseudo-events.
|
||||
* <p>
|
||||
* When alter table event for rename is seen at the event dispatcher, two instances of
|
||||
* this class are created. One instance is with a drop table pseudo-event and the other
|
||||
* is with a create table pseudo-event. Both instances have the same reference to
|
||||
* processing state and actual alter table event. These events are dispatched to
|
||||
* processing state i.e., {@link RenameEventState} and
|
||||
* context i.e., {@link DerivedMetastoreEventContext}. These events are dispatched to
|
||||
* appropriate DbEventExecutor.
|
||||
* <p>
|
||||
* When a TableProcessor's event processing encounter an instance of
|
||||
@@ -57,11 +58,7 @@ import static org.apache.impala.catalog.Table.TBL_EVENTS_PROCESS_DURATION;
|
||||
*
|
||||
* @see org.apache.impala.catalog.events.TableEventExecutor.TableProcessor
|
||||
*/
|
||||
public class RenameTableBarrierEvent extends MetastoreTableEvent
|
||||
implements DerivedMetastoreEvent {
|
||||
// alter table event with rename true
|
||||
private final AlterTableEvent actualEvent_;
|
||||
|
||||
public class RenameTableBarrierEvent extends DerivedMetastoreTableEvent {
|
||||
// Drop table or Create table event
|
||||
private final MetastoreTableEvent pseudoEvent_;
|
||||
|
||||
@@ -140,17 +137,15 @@ public class RenameTableBarrierEvent extends MetastoreTableEvent
|
||||
}
|
||||
}
|
||||
|
||||
RenameTableBarrierEvent(AlterTableEvent actualEvent,
|
||||
RenameTableBarrierEvent(DerivedMetastoreEventContext context,
|
||||
MetastoreTableEvent pseudoEvent, RenameEventState state) {
|
||||
super(pseudoEvent.catalogOpExecutor_, pseudoEvent.metrics_, pseudoEvent.event_);
|
||||
Preconditions.checkArgument(actualEvent != null);
|
||||
super(context, pseudoEvent.event_);
|
||||
Preconditions.checkArgument(state != null);
|
||||
Preconditions.checkArgument(
|
||||
pseudoEvent.getEventType() == MetastoreEventType.DROP_TABLE ||
|
||||
pseudoEvent.getEventType() == MetastoreEventType.CREATE_TABLE);
|
||||
actualEvent_ = actualEvent;
|
||||
pseudoEvent_ = pseudoEvent;
|
||||
this.state_ = state;
|
||||
state_ = state;
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -159,9 +154,7 @@ public class RenameTableBarrierEvent extends MetastoreTableEvent
|
||||
* @return True if the event can be processed. False otherwise
|
||||
*/
|
||||
boolean canProcess() {
|
||||
if (getEventType() == MetastoreEventType.DROP_TABLE) {
|
||||
return !state_.isDropProcessed();
|
||||
}
|
||||
if (getEventType() == MetastoreEventType.DROP_TABLE) return !state_.isDropProcessed();
|
||||
return state_.isDropProcessed() && !state_.isCreateProcessed();
|
||||
}
|
||||
|
||||
@@ -170,7 +163,7 @@ public class RenameTableBarrierEvent extends MetastoreTableEvent
|
||||
if (state_.isSkipped()) {
|
||||
// Update the skipped metrics if both events are skipped
|
||||
metrics_.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).inc();
|
||||
actualEvent_.debugLog("Incremented skipped metric to {}",
|
||||
getActualEvent().debugLog("Incremented skipped metric to {}",
|
||||
metrics_.getCounter(MetastoreEventsProcessor.EVENTS_SKIPPED_METRIC).getCount());
|
||||
}
|
||||
if (state_.isProcessed()) {
|
||||
@@ -186,7 +179,7 @@ public class RenameTableBarrierEvent extends MetastoreTableEvent
|
||||
@Override
|
||||
public void processIfEnabled() throws CatalogException {
|
||||
Preconditions.checkState(canProcess());
|
||||
if (actualEvent_.isEventProcessingDisabled()) {
|
||||
if (getActualEvent().isEventProcessingDisabled()) {
|
||||
updateStatus(true);
|
||||
return;
|
||||
}
|
||||
@@ -196,9 +189,7 @@ public class RenameTableBarrierEvent extends MetastoreTableEvent
|
||||
DebugUtils.EVENT_PROCESSING_DELAY);
|
||||
}
|
||||
process();
|
||||
if (state_.isProcessed()) {
|
||||
actualEvent_.injectErrorIfNeeded();
|
||||
}
|
||||
if (state_.isProcessed()) getActualEvent().injectErrorIfNeeded();
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -206,9 +197,7 @@ public class RenameTableBarrierEvent extends MetastoreTableEvent
|
||||
Timer.Context context = null;
|
||||
Db db = catalog_.getDb(getDbName());
|
||||
Table table = null;
|
||||
if (db != null) {
|
||||
table = db.getTable(getTableName());
|
||||
}
|
||||
if (db != null) table = db.getTable(getTableName());
|
||||
if (table != null) {
|
||||
context = table.getMetrics().getTimer(TBL_EVENTS_PROCESS_DURATION).time();
|
||||
}
|
||||
@@ -219,7 +208,7 @@ public class RenameTableBarrierEvent extends MetastoreTableEvent
|
||||
if (table != null && catalogOpExecutor_.removeTableIfNotAddedLater(getEventId(),
|
||||
getDbName(), getTableName(), new Reference<>())) {
|
||||
skipped = false;
|
||||
actualEvent_.infoLog("Successfully removed table {}", fqTableName);
|
||||
getActualEvent().infoLog("Successfully removed table {}", fqTableName);
|
||||
metrics_.getCounter(MetastoreEventsProcessor.NUMBER_OF_TABLES_REMOVED).inc();
|
||||
}
|
||||
} else {
|
||||
@@ -227,14 +216,12 @@ public class RenameTableBarrierEvent extends MetastoreTableEvent
|
||||
if (db != null && catalogOpExecutor_.addTableIfNotRemovedLater(getEventId(),
|
||||
pseudoEvent_.getTable())) {
|
||||
skipped = false;
|
||||
actualEvent_.infoLog("Successfully added table {}", fqTableName);
|
||||
getActualEvent().infoLog("Successfully added table {}", fqTableName);
|
||||
metrics_.getCounter(MetastoreEventsProcessor.NUMBER_OF_TABLES_ADDED).inc();
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
if (context != null) {
|
||||
context.stop();
|
||||
}
|
||||
if (context != null) context.stop();
|
||||
}
|
||||
updateStatus(skipped);
|
||||
}
|
||||
@@ -270,6 +257,6 @@ public class RenameTableBarrierEvent extends MetastoreTableEvent
|
||||
|
||||
@Override
|
||||
public String getEventDesc() {
|
||||
return actualEvent_.getEventDesc() + " pseudo-event " + getEventType();
|
||||
return getActualEvent().getEventDesc() + " pseudo-event " + getEventType();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -30,10 +30,10 @@ import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.DropTableEvent;
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.DerivedMetastoreTableEvent;
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEvent;
|
||||
import org.apache.impala.catalog.events.MetastoreEvents.MetastoreEventType;
|
||||
import org.apache.impala.catalog.events.MetastoreEventsProcessor.EventProcessorStatus;
|
||||
import org.apache.impala.common.PrintUtils;
|
||||
import org.apache.impala.service.BackendConfig;
|
||||
import org.apache.impala.util.ClassUtil;
|
||||
import org.apache.impala.util.ThreadNameAnnotator;
|
||||
@@ -104,11 +104,24 @@ public class TableEventExecutor {
|
||||
/**
|
||||
* Indicates whether TableProcessor is terminating. Events are processed only when
|
||||
* isTerminating_ is false.
|
||||
* Event processing {@link TableProcessor#process()} and deletion
|
||||
* {@link TableEventExecutor#deleteTableProcessor(String)} are invoked from different
|
||||
* threads. {@link TableEventExecutor#deleteTableProcessor(String)} acquires
|
||||
* processorLock_ to immediately set isTerminating_ to true, thereby preventing any
|
||||
* further event processing.
|
||||
* <p>
|
||||
* TableProcessor event queueing {@link TableProcessor#enqueue(MetastoreEvent)} and
|
||||
* processing {@link TableProcessor#process()} are invoked from different threads.
|
||||
* {@link TableProcessor#enqueue(MetastoreEvent)} acquires processorLock_ to
|
||||
* atomically update the event queue, increment outstanding event count so that
|
||||
* {@link TableProcessor#process()} gets the consistent view of them together.
|
||||
* <p>
|
||||
* TableProcessor deletion {@link TableEventExecutor#deleteTableProcessor(String)}
|
||||
* and processing {@link TableProcessor#process()} are invoked from different threads.
|
||||
* {@link TableEventExecutor#deleteTableProcessor(String)} acquires processorLock_ to
|
||||
* immediately set isTerminating_ to true, thereby preventing any further event
|
||||
* processing in {@link TableProcessor#process()}.
|
||||
* <p>
|
||||
* TableProcessor event queueing {@link TableProcessor#enqueue(MetastoreEvent)}
|
||||
* and deletion {@link TableEventExecutor#deleteTableProcessor(String)} are invoked
|
||||
* from DbProcessor in same thread(upon TableProcessor idle timeout) or in different
|
||||
* threads(forcibly clear). But their invocation is mutually exclusive because
|
||||
* DbProcessor protects with its processorLock_.
|
||||
* <p>
|
||||
* {@link TableProcessor#process()} acquires processorLock_ in following cases:
|
||||
* <ul>
|
||||
@@ -122,10 +135,6 @@ public class TableEventExecutor {
|
||||
* {@link DbEventExecutor#clear()} or {@link DbEventExecutor#stop()} is invoked,
|
||||
* all associated DbProcessors are forcibly cleared and removed. During that process,
|
||||
* each DbProcessor also forcefully deletes its TableProcessors.
|
||||
* <p>
|
||||
* Lock contention is unlikely at the most frequent call site (i.e., within
|
||||
* {@link TableProcessor#process()}, since lock acquired in deleteTableProcessor() is
|
||||
* held only once.
|
||||
*/
|
||||
private final Object processorLock_ = new Object();
|
||||
private volatile boolean isTerminating_ = false;
|
||||
@@ -158,16 +167,23 @@ public class TableEventExecutor {
|
||||
return events_.isEmpty();
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets Metastore event processor
|
||||
* @return
|
||||
*/
|
||||
private MetastoreEventsProcessor getEventProcessor() {
|
||||
return tableEventExecutor_.eventProcessor_;
|
||||
}
|
||||
|
||||
/**
|
||||
* Queue the event to the TableProcessor for processing.
|
||||
* @param event
|
||||
*/
|
||||
void enqueue(MetastoreEvent event) {
|
||||
if (tableEventExecutor_.eventProcessor_.getStatus() !=
|
||||
EventProcessorStatus.ACTIVE) {
|
||||
MetastoreEventsProcessor eventProcessor = getEventProcessor();
|
||||
if (!eventProcessor.canProcessEventInCurrentStatus()) {
|
||||
event.warnLog("Event is not queued to executor: {} since status is {}",
|
||||
tableEventExecutor_.name_,
|
||||
tableEventExecutor_.eventProcessor_.getStatus());
|
||||
tableEventExecutor_.name_, eventProcessor.getStatus());
|
||||
return;
|
||||
}
|
||||
lastEventQueuedTime_ = System.currentTimeMillis();
|
||||
@@ -176,8 +192,11 @@ public class TableEventExecutor {
|
||||
} else if (event.getEventType() == MetastoreEventType.DROP_TABLE) {
|
||||
skipEventId_.set(event.getEventId());
|
||||
}
|
||||
synchronized (processorLock_) {
|
||||
Preconditions.checkState(!isTerminating());
|
||||
events_.offer(event);
|
||||
tableEventExecutor_.incrOutstandingEventCount();
|
||||
}
|
||||
event.debugLog("Enqueued for table: {} on executor: {}", fqTableName_,
|
||||
tableEventExecutor_.name_);
|
||||
}
|
||||
@@ -210,13 +229,10 @@ public class TableEventExecutor {
|
||||
/**
|
||||
* Skip the metastore event from processing if possible.
|
||||
* @param event Metastore event
|
||||
* @param dropTableEventId Drop table event id if drop table event or rename table
|
||||
* barrier with drop table event is queued for processing.
|
||||
* Else -1.
|
||||
* @return True if event is skipped. Else false.
|
||||
*/
|
||||
private boolean skipEventIfPossible(MetastoreEvent event, long dropTableEventId) {
|
||||
if (event.getEventId() >= dropTableEventId || event instanceof DbBarrierEvent ||
|
||||
private boolean skipEventIfPossible(MetastoreEvent event) {
|
||||
if (event.getEventId() >= skipEventId_.get() || event instanceof DbBarrierEvent ||
|
||||
event instanceof RenameTableBarrierEvent) {
|
||||
return false;
|
||||
}
|
||||
@@ -228,26 +244,46 @@ public class TableEventExecutor {
|
||||
}
|
||||
|
||||
/**
|
||||
* Process the events on the TableProcessor. It is invoked from TableEventExecutor's
|
||||
* thread periodically to process events for the TableProcessor.
|
||||
* Method invoked after processing the metastore event.
|
||||
* @param event Metastore event
|
||||
*/
|
||||
private void postProcessEvent(MetastoreEvent event) {
|
||||
if (event instanceof DbBarrierEvent) return;
|
||||
lastProcessedEventId_ = event.getEventId();
|
||||
MetastoreEventsProcessor eventProcessor = getEventProcessor();
|
||||
boolean removeFromInProgressLog = true;
|
||||
if (event instanceof DerivedMetastoreTableEvent) {
|
||||
DerivedMetastoreTableEvent derivedEvent = (DerivedMetastoreTableEvent) event;
|
||||
derivedEvent.markProcessed();
|
||||
removeFromInProgressLog = derivedEvent.isAllDerivedEventsProcessed();
|
||||
}
|
||||
if (removeFromInProgressLog) {
|
||||
eventProcessor.getEventExecutorService()
|
||||
.removeFromInProgressLog(event.getEventId());
|
||||
}
|
||||
if (event.isDropEvent()) {
|
||||
eventProcessor.getDeleteEventLog().removeEvent(event.getEventId());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Process the given metastore event.
|
||||
* @param event Metastore event
|
||||
* @return True if event is processed. False otherwise
|
||||
* @throws Exception
|
||||
*/
|
||||
private void process() throws Exception {
|
||||
MetastoreEvent event;
|
||||
String annotation = "Processing %s for table: " + fqTableName_;
|
||||
long skipEventId = skipEventId_.get();
|
||||
while ((event = events_.peek()) != null) {
|
||||
if (isTerminating()) return;
|
||||
if (tableEventExecutor_.eventProcessor_.getStatus() !=
|
||||
EventProcessorStatus.ACTIVE) {
|
||||
private boolean processEvent(MetastoreEvent event) throws Exception {
|
||||
if (isTerminating()) return false;
|
||||
MetastoreEventsProcessor eventProcessor = getEventProcessor();
|
||||
if (!eventProcessor.canProcessEventInCurrentStatus()) {
|
||||
LOG.warn("Event processing is skipped for executor: {} since status is {}",
|
||||
tableEventExecutor_.name_, tableEventExecutor_.eventProcessor_.getStatus());
|
||||
return;
|
||||
tableEventExecutor_.name_, eventProcessor.getStatus());
|
||||
return false;
|
||||
}
|
||||
boolean isRenameTableBarrier = event instanceof RenameTableBarrierEvent;
|
||||
if (isRenameTableBarrier && !((RenameTableBarrierEvent) event).canProcess()) {
|
||||
event.traceLog("Rename table barrier waiting for table: {}", fqTableName_);
|
||||
return;
|
||||
return false;
|
||||
}
|
||||
boolean isDbEvent = event instanceof DbBarrierEvent;
|
||||
if (isDbEvent && event.getEventId() != lastProcessedEventId_) {
|
||||
@@ -255,22 +291,25 @@ public class TableEventExecutor {
|
||||
((DbBarrierEvent) event).proceed();
|
||||
lastProcessedEventId_ = event.getEventId();
|
||||
}
|
||||
if (!skipEventIfPossible(event, skipEventId)) {
|
||||
if (!skipEventIfPossible(event)) {
|
||||
if (isDbEvent) {
|
||||
if (!((DbBarrierEvent) event).isProcessed()) {
|
||||
if (!((DbBarrierEvent) event).isAllDerivedEventsProcessed()) {
|
||||
// Waiting for the db event to be processed
|
||||
event.traceLog("DB barrier waiting for table: {}", fqTableName_);
|
||||
return;
|
||||
return false;
|
||||
}
|
||||
} else {
|
||||
try (ThreadNameAnnotator tna = new ThreadNameAnnotator(
|
||||
String.format(annotation, event.getEventDesc()))) {
|
||||
String.format("Processing %s for table: %s", event.getEventDesc(),
|
||||
fqTableName_))) {
|
||||
long processingStartTime = System.currentTimeMillis();
|
||||
event.processIfEnabled();
|
||||
event.infoLog("Scheduling delay: {}, Process time: {}",
|
||||
PrintUtils.printTimeMs(processingStartTime - event.getDispatchTime()),
|
||||
PrintUtils.printTimeMs(System.currentTimeMillis() - processingStartTime));
|
||||
} catch (Exception processingEx) {
|
||||
try {
|
||||
if (!event.onFailure(processingEx)) {
|
||||
throw processingEx;
|
||||
}
|
||||
if (!event.onFailure(processingEx)) throw processingEx;
|
||||
} catch (Exception onFailureEx) {
|
||||
event.errorLog("Failed to handle event processing failure for table: {}",
|
||||
fqTableName_, onFailureEx);
|
||||
@@ -279,19 +318,36 @@ public class TableEventExecutor {
|
||||
}
|
||||
}
|
||||
}
|
||||
lastProcessedEventId_ = event.getEventId();
|
||||
event.infoLog("Processed for table: {},", fqTableName_);
|
||||
if (event.isDropEvent()) {
|
||||
tableEventExecutor_.eventProcessor_.getDeleteEventLog()
|
||||
.removeEvent(event.getEventId());
|
||||
return true;
|
||||
}
|
||||
|
||||
/**
|
||||
* Process the events on the TableProcessor. It is invoked from TableEventExecutor's
|
||||
* thread periodically to process events for the TableProcessor.
|
||||
* @throws EventProcessException
|
||||
*/
|
||||
private void process() throws EventProcessException {
|
||||
MetastoreEvent event;
|
||||
while ((event = events_.peek()) != null) {
|
||||
try {
|
||||
boolean isProcessed = processEvent(event);
|
||||
if (!isProcessed) return;
|
||||
postProcessEvent(event);
|
||||
synchronized (processorLock_) {
|
||||
if (isTerminating()) return;
|
||||
Preconditions.checkState(events_.poll() == event);
|
||||
tableEventExecutor_.decrOutstandingEventCount(1);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
// Throwing EventProcessException triggers global invalidates metadata without
|
||||
// user intervention iff invalidate_global_metadata_on_event_processing_failure
|
||||
// flag is true. Otherwise, user has to explicitly issue invalidate metadata.
|
||||
// Invalidate metadata resets catalog instance that clears EventExecutorService.
|
||||
// And EventExecutorService inherently clears all DbEventExecutor and
|
||||
// TableEventExecutor thereby removing all DbProcessors and TableProcessors.
|
||||
throw new EventProcessException(event.getEvent(), e);
|
||||
}
|
||||
}
|
||||
skipEventId_.compareAndSet(skipEventId, -1);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -414,7 +470,6 @@ public class TableEventExecutor {
|
||||
// other places if isTerminating_ becomes true.
|
||||
decrOutstandingEventCount(tableProcessor.events_.size());
|
||||
tableProcessor.events_.clear();
|
||||
tableProcessor.skipEventId_.set(-1);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -426,13 +481,12 @@ public class TableEventExecutor {
|
||||
try {
|
||||
for (Map.Entry<String, TableProcessor> entry : tableProcessors_.entrySet()) {
|
||||
TableProcessor tableProcessor = entry.getValue();
|
||||
if (eventProcessor_.getStatus() != EventProcessorStatus.ACTIVE) {
|
||||
break;
|
||||
}
|
||||
if (!eventProcessor_.canProcessEventInCurrentStatus()) break;
|
||||
tableProcessor.process();
|
||||
}
|
||||
} catch (Exception e) {
|
||||
eventProcessor_.handleEventProcessException(e);
|
||||
} catch (EventProcessException e) {
|
||||
LOG.error("Exception occurred for executor: {}", name_);
|
||||
eventProcessor_.handleEventProcessException(e.getException(), e.getEvent());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -162,16 +162,17 @@ public class EventExecutorServiceTest {
|
||||
}
|
||||
}
|
||||
|
||||
private void insertIntoTransactionalTable(MetaStoreClient msClient, Table table,
|
||||
List<String> partVal, long txnId, long writeId) throws TException, IOException {
|
||||
private void insertIntoTransactionalTable(MetaStoreClient msClient, String dbName,
|
||||
String tableName, List<String> partVal, long txnId, long writeId) throws TException,
|
||||
IOException {
|
||||
org.apache.hadoop.hive.metastore.api.Table table = msClient.getHiveClient().getTable(
|
||||
dbName, tableName);
|
||||
Partition partition = null;
|
||||
if (!CollectionUtils.isEmpty(partVal)) {
|
||||
partition = msClient.getHiveClient().getPartition(table.getDb().getName(),
|
||||
table.getName(), partVal);
|
||||
partition = msClient.getHiveClient().getPartition(dbName, tableName, partVal);
|
||||
}
|
||||
MetastoreApiTestUtils.simulateInsertIntoTransactionalTableFromFS(
|
||||
catalog_.getMetaStoreClient(), table.getMetaStoreTable(), partition, 1, txnId,
|
||||
writeId);
|
||||
catalog_.getMetaStoreClient(), table, partition, 1, txnId, writeId);
|
||||
}
|
||||
|
||||
private void alterTableRename(String dbName, String tblName, String newDbName,
|
||||
@@ -240,8 +241,7 @@ public class EventExecutorServiceTest {
|
||||
*/
|
||||
@Test
|
||||
public void testAssignAndUnAssignExecutors() throws Exception {
|
||||
EventExecutorService eventExecutorService = new EventExecutorService(eventsProcessor_,
|
||||
2, 3);
|
||||
EventExecutorService eventExecutorService = createEventExecutorService(2, 3);
|
||||
List<DbEventExecutor> dbEventExecutorList =
|
||||
eventExecutorService.getDbEventExecutors();
|
||||
assertEquals(2, dbEventExecutorList.size());
|
||||
@@ -252,8 +252,6 @@ public class EventExecutorServiceTest {
|
||||
TableEventExecutor tableExecutor1 = dbExecutor1.getTableEventExecutors().get(0);
|
||||
TableEventExecutor tableExecutor2 = dbExecutor1.getTableEventExecutors().get(1);
|
||||
TableEventExecutor tableExecutor3 = dbExecutor2.getTableEventExecutors().get(0);
|
||||
eventExecutorService.start();
|
||||
eventsProcessor_.setEventExecutorService(eventExecutorService);
|
||||
assertEquals(0, dbExecutor1.getDbCount());
|
||||
assertEquals(0, dbExecutor2.getDbCount());
|
||||
assertEquals(0, tableExecutor1.getTableCount());
|
||||
@@ -319,12 +317,7 @@ public class EventExecutorServiceTest {
|
||||
shutDownEventExecutorService(eventExecutorService);
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests clear executors
|
||||
* @throws Exception
|
||||
*/
|
||||
@Test
|
||||
public void testClearExecutors() throws Exception {
|
||||
private void clearOrShutEventExecutorService(boolean isShutDown) throws Exception {
|
||||
EventExecutorService eventExecutorService = new EventExecutorService(eventsProcessor_,
|
||||
2, 2);
|
||||
List<DbEventExecutor> dbEventExecutorList =
|
||||
@@ -336,7 +329,9 @@ public class EventExecutorServiceTest {
|
||||
assertEquals(2, dbExecutor2.getTableEventExecutors().size());
|
||||
TableEventExecutor tableExecutor1 = dbExecutor1.getTableEventExecutors().get(0);
|
||||
TableEventExecutor tableExecutor2 = dbExecutor1.getTableEventExecutors().get(1);
|
||||
eventExecutorService.start();
|
||||
TableEventExecutor tableExecutor3 = dbExecutor2.getTableEventExecutors().get(0);
|
||||
TableEventExecutor tableExecutor4 = dbExecutor2.getTableEventExecutors().get(1);
|
||||
eventExecutorService.setStatus(EventExecutorService.EventExecutorStatus.ACTIVE);
|
||||
createDatabase(DB_NAME1);
|
||||
createDatabase(DB_NAME2);
|
||||
for (int i = 0; i < 10; i++) {
|
||||
@@ -349,12 +344,37 @@ public class EventExecutorServiceTest {
|
||||
for (MetastoreEvent event : metastoreEvents) {
|
||||
eventExecutorService.dispatch(event);
|
||||
}
|
||||
for (DbEventExecutor executor : dbEventExecutorList) {
|
||||
executor.process();
|
||||
}
|
||||
assertEquals(1, dbExecutor1.getDbCount());
|
||||
assertEquals(1, dbExecutor2.getDbCount());
|
||||
assertEquals(5, tableExecutor1.getTableCount());
|
||||
assertEquals(5, tableExecutor2.getTableCount());
|
||||
assertEquals(5, tableExecutor3.getTableCount());
|
||||
assertEquals(5, tableExecutor4.getTableCount());
|
||||
if (isShutDown) {
|
||||
eventExecutorService.shutdown(false);
|
||||
} else {
|
||||
eventExecutorService.clear();
|
||||
}
|
||||
assertEquals(0, dbExecutor1.getDbCount());
|
||||
assertEquals(0, dbExecutor2.getDbCount());
|
||||
assertEquals(0, dbExecutor1.getOutstandingEventCount());
|
||||
assertEquals(0, tableExecutor1.getTableCount());
|
||||
assertEquals(0, tableExecutor2.getTableCount());
|
||||
eventExecutorService.shutdown(true);
|
||||
assertEquals(0, tableExecutor3.getTableCount());
|
||||
assertEquals(0, tableExecutor4.getTableCount());
|
||||
if (!isShutDown) eventExecutorService.shutdown(true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests clear executors
|
||||
* @throws Exception
|
||||
*/
|
||||
@Test
|
||||
public void testClearExecutors() throws Exception {
|
||||
clearOrShutEventExecutorService(false);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -363,35 +383,7 @@ public class EventExecutorServiceTest {
|
||||
*/
|
||||
@Test
|
||||
public void testForceShutdownExecutors() throws Exception {
|
||||
EventExecutorService eventExecutorService = new EventExecutorService(eventsProcessor_,
|
||||
2, 2);
|
||||
List<DbEventExecutor> dbEventExecutorList =
|
||||
eventExecutorService.getDbEventExecutors();
|
||||
assertEquals(2, dbEventExecutorList.size());
|
||||
DbEventExecutor dbExecutor1 = dbEventExecutorList.get(0);
|
||||
DbEventExecutor dbExecutor2 = dbEventExecutorList.get(1);
|
||||
assertEquals(2, dbExecutor1.getTableEventExecutors().size());
|
||||
assertEquals(2, dbExecutor2.getTableEventExecutors().size());
|
||||
TableEventExecutor tableExecutor1 = dbExecutor1.getTableEventExecutors().get(0);
|
||||
TableEventExecutor tableExecutor2 = dbExecutor1.getTableEventExecutors().get(1);
|
||||
eventExecutorService.start();
|
||||
createDatabase(DB_NAME1);
|
||||
createDatabase(DB_NAME2);
|
||||
for (int i = 0; i < 10; i++) {
|
||||
createTable(DB_NAME1, "t"+i);
|
||||
createTable(DB_NAME2, "t"+i);
|
||||
}
|
||||
List<MetastoreEvent> metastoreEvents = eventsProcessor_.getEventsFactory()
|
||||
.getFilteredEvents(eventsProcessor_.getNextMetastoreEvents(),
|
||||
eventsProcessor_.getMetrics());
|
||||
for (MetastoreEvent event : metastoreEvents) {
|
||||
eventExecutorService.dispatch(event);
|
||||
}
|
||||
eventExecutorService.shutdown(false);
|
||||
assertEquals(0, dbExecutor1.getDbCount());
|
||||
assertEquals(0, dbExecutor1.getOutstandingEventCount());
|
||||
assertEquals(0, tableExecutor1.getTableCount());
|
||||
assertEquals(0, tableExecutor2.getTableCount());
|
||||
clearOrShutEventExecutorService(true);
|
||||
}
|
||||
|
||||
private void renameTableTest(String srcDbName, String srcTableName, String targetDbName,
|
||||
@@ -589,7 +581,7 @@ public class EventExecutorServiceTest {
|
||||
txnId = MetastoreShim.openTransaction(msClient.getHiveClient());
|
||||
writeId = MetastoreShim.allocateTableWriteId(msClient.getHiveClient(), txnId,
|
||||
DB_NAME1, t1);
|
||||
insertIntoTransactionalTable(msClient, t1Table, null, txnId, writeId);
|
||||
insertIntoTransactionalTable(msClient, DB_NAME1, t1, null, txnId, writeId);
|
||||
MetastoreShim.commitTransaction(msClient.getHiveClient(), txnId);
|
||||
eventsProcessor_.processEvents();
|
||||
ValidWriteIdList writeIdList = t1Table.getValidWriteIds();
|
||||
@@ -599,8 +591,10 @@ public class EventExecutorServiceTest {
|
||||
txnId = MetastoreShim.openTransaction(msClient.getHiveClient());
|
||||
writeId = MetastoreShim.allocateTableWriteId(msClient.getHiveClient(), txnId,
|
||||
DB_NAME1, t2);
|
||||
insertIntoTransactionalTable(msClient, t2Table, Arrays.asList("1"), txnId, writeId);
|
||||
insertIntoTransactionalTable(msClient, t2Table, Arrays.asList("2"), txnId, writeId);
|
||||
insertIntoTransactionalTable(msClient, DB_NAME1, t2, Arrays.asList("1"), txnId,
|
||||
writeId);
|
||||
insertIntoTransactionalTable(msClient, DB_NAME1, t2, Arrays.asList("2"), txnId,
|
||||
writeId);
|
||||
MetastoreShim.commitTransaction(msClient.getHiveClient(), txnId);
|
||||
eventsProcessor_.processEvents();
|
||||
writeIdList = t2Table.getValidWriteIds();
|
||||
@@ -610,14 +604,14 @@ public class EventExecutorServiceTest {
|
||||
txnId = MetastoreShim.openTransaction(msClient.getHiveClient());
|
||||
long writeId1 = MetastoreShim.allocateTableWriteId(msClient.getHiveClient(), txnId,
|
||||
DB_NAME1, t1);
|
||||
insertIntoTransactionalTable(msClient, t1Table, null, txnId, writeId1);
|
||||
insertIntoTransactionalTable(msClient, DB_NAME1, t1, null, txnId, writeId1);
|
||||
long writeId2 = MetastoreShim.allocateTableWriteId(msClient.getHiveClient(), txnId,
|
||||
DB_NAME1, t2);
|
||||
insertIntoTransactionalTable(msClient, t2Table, Arrays.asList("2"), txnId,
|
||||
insertIntoTransactionalTable(msClient, DB_NAME1, t2, Arrays.asList("2"), txnId,
|
||||
writeId2);
|
||||
long writeId3 = MetastoreShim.allocateTableWriteId(msClient.getHiveClient(), txnId,
|
||||
DB_NAME2, t3);
|
||||
insertIntoTransactionalTable(msClient, t3Table, Arrays.asList("3"), txnId,
|
||||
insertIntoTransactionalTable(msClient, DB_NAME2, t3, Arrays.asList("3"), txnId,
|
||||
writeId3);
|
||||
if (abortTxn) {
|
||||
MetastoreShim.abortTransaction(msClient.getHiveClient(), txnId);
|
||||
@@ -770,7 +764,7 @@ public class EventExecutorServiceTest {
|
||||
eventExecutorService.dispatch(event);
|
||||
}
|
||||
processEventsSynchronously(eventExecutorService);
|
||||
eventExecutorService.setStatus(EventExecutorService.EventExecutorStatus.STOPPED);
|
||||
eventExecutorService.shutdown(true);
|
||||
// Skips all the events i.e., create database events(DB_NAME1 and DB_NAME2), create
|
||||
// table event(DB_NAME1.t1), alter database event for DB_NAME1, rename table from
|
||||
// DB_NAME1.t1 to DB_NAME2.t2 event, drop database event for DB_NAME1, drop table
|
||||
@@ -788,4 +782,135 @@ public class EventExecutorServiceTest {
|
||||
assertEquals(tablesRemovedMetric,
|
||||
metrics.getCounter(MetastoreEventsProcessor.NUMBER_OF_TABLES_REMOVED).getCount());
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests greatest synced event id
|
||||
* @throws Exception
|
||||
*/
|
||||
@Test
|
||||
public void testGreatestSyncedEvent() throws Exception {
|
||||
EventExecutorService eventExecutorService = createEventExecutorService(1, 1);
|
||||
assertEquals(0, eventExecutorService.getOutstandingEventCount());
|
||||
long lastSyncedEventId = eventsProcessor_.getLastSyncedEventId();
|
||||
assertEquals(lastSyncedEventId, eventExecutorService.getGreatestSyncedEventId());
|
||||
assertEquals(eventsProcessor_.getLastSyncedEventTime(),
|
||||
eventExecutorService.getGreatestSyncedEventTime());
|
||||
createDatabase(DB_NAME1);
|
||||
createTable(DB_NAME1, "t1");
|
||||
eventsProcessor_.processEvents();
|
||||
Table table = catalog_.getOrLoadTable(DB_NAME1, "t1", "test", null);
|
||||
assertTrue("Table should have been loaded.", table instanceof HdfsTable);
|
||||
assertEquals(lastSyncedEventId + 2, eventsProcessor_.getLastSyncedEventId());
|
||||
assertEquals(eventsProcessor_.getLastSyncedEventId(),
|
||||
eventExecutorService.getGreatestSyncedEventId());
|
||||
assertEquals(eventsProcessor_.getLastSyncedEventTime(),
|
||||
eventExecutorService.getGreatestSyncedEventTime());
|
||||
shutDownEventExecutorService(eventExecutorService);
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests in-progress and processed event logs
|
||||
* @throws Exception
|
||||
*/
|
||||
@Test
|
||||
public void testEventLogs() throws Exception {
|
||||
EventExecutorService eventExecutorService = new EventExecutorService(eventsProcessor_,
|
||||
2, 1);
|
||||
eventExecutorService.setStatus(EventExecutorService.EventExecutorStatus.ACTIVE);
|
||||
long lastSyncedEventId = eventsProcessor_.getLastSyncedEventId();
|
||||
createDatabase(DB_NAME1);
|
||||
String t1 = "t1";
|
||||
String t2 = "t2";
|
||||
createTransactionalTable(DB_NAME1, t1, false);
|
||||
try (MetaStoreClient msClient = catalog_.getMetaStoreClient()) {
|
||||
// Commit a transaction on t1
|
||||
long txnId = MetastoreShim.openTransaction(msClient.getHiveClient());
|
||||
long writeId = MetastoreShim.allocateTableWriteId(msClient.getHiveClient(), txnId,
|
||||
DB_NAME1, t1);
|
||||
MetastoreShim.commitTransaction(msClient.getHiveClient(), txnId);
|
||||
}
|
||||
createDatabase(DB_NAME2);
|
||||
createTable(DB_NAME2, t2);
|
||||
// Order of metastore events(6) generated are:
|
||||
// create db1, create table t1, alloc write id for t1, commit txn for t1, create db2,
|
||||
// create table t2
|
||||
// Note: open txn event is generated before alloc write id for t1 event. But it is not
|
||||
// fetched from HMS by default
|
||||
List<MetastoreEvent> metastoreEvents = eventsProcessor_.getEventsFactory()
|
||||
.getFilteredEvents(eventsProcessor_.getNextMetastoreEvents(),
|
||||
eventsProcessor_.getMetrics());
|
||||
assertEquals(6, metastoreEvents.size());
|
||||
assertEquals(0, eventExecutorService.getInProgressLog().size());
|
||||
assertEquals(1, eventExecutorService.getProcessedLog().size());
|
||||
assertEquals(lastSyncedEventId, eventExecutorService.getGreatestSyncedEventId());
|
||||
assertEquals(0, eventExecutorService.getPendingEventCount(lastSyncedEventId));
|
||||
|
||||
long lastEventId = metastoreEvents.get(5).getEventId();
|
||||
assertEquals(7, eventExecutorService.getPendingEventCount(lastEventId));
|
||||
|
||||
for (MetastoreEvent event : metastoreEvents) {
|
||||
eventExecutorService.dispatch(event);
|
||||
}
|
||||
assertEquals(6, eventExecutorService.getInProgressLog().size());
|
||||
assertEquals(1, eventExecutorService.getProcessedLog().size());
|
||||
assertEquals(lastSyncedEventId, eventExecutorService.getGreatestSyncedEventId());
|
||||
// Pending event count till last(6th) event
|
||||
assertEquals(6, eventExecutorService.getPendingEventCount(lastEventId));
|
||||
// Pending event count if 100 more events are to process beyond last event
|
||||
assertEquals(6 + 100, eventExecutorService.getPendingEventCount(lastEventId + 100));
|
||||
|
||||
// Remove the 5th event
|
||||
eventExecutorService.removeFromInProgressLog(metastoreEvents.get(4).getEventId());
|
||||
assertEquals(5, eventExecutorService.getInProgressLog().size());
|
||||
assertEquals(2, eventExecutorService.getProcessedLog().size());
|
||||
// Greatest synced event id shouldn't change
|
||||
assertEquals(lastSyncedEventId, eventExecutorService.getGreatestSyncedEventId());
|
||||
// Pending event count till last(6th) event
|
||||
assertEquals(5, eventExecutorService.getPendingEventCount(lastEventId));
|
||||
// Pending event count till 5th event
|
||||
assertEquals(4, eventExecutorService.getPendingEventCount(
|
||||
metastoreEvents.get(4).getEventId()));
|
||||
// Pending event count if 100 more events are to process beyond last event
|
||||
assertEquals(5 + 100, eventExecutorService.getPendingEventCount(lastEventId + 100));
|
||||
|
||||
// Remove the 1st event
|
||||
eventExecutorService.removeFromInProgressLog(metastoreEvents.get(0).getEventId());
|
||||
assertEquals(4, eventExecutorService.getInProgressLog().size());
|
||||
assertEquals(2, eventExecutorService.getProcessedLog().size());
|
||||
// Greatest synced event id should become the 1st event's id
|
||||
lastSyncedEventId = metastoreEvents.get(0).getEventId();
|
||||
assertEquals(lastSyncedEventId, eventExecutorService.getGreatestSyncedEventId());
|
||||
assertEquals(4, eventExecutorService.getPendingEventCount(lastEventId));
|
||||
|
||||
// Remove the 2nd event
|
||||
eventExecutorService.removeFromInProgressLog(metastoreEvents.get(1).getEventId());
|
||||
assertEquals(3, eventExecutorService.getInProgressLog().size());
|
||||
assertEquals(2, eventExecutorService.getProcessedLog().size());
|
||||
// Greatest synced event id should become the 2nd event's id
|
||||
lastSyncedEventId = metastoreEvents.get(1).getEventId();
|
||||
assertEquals(lastSyncedEventId, eventExecutorService.getGreatestSyncedEventId());
|
||||
assertEquals(3, eventExecutorService.getPendingEventCount(lastEventId));
|
||||
|
||||
// Remove the 3rd event
|
||||
eventExecutorService.removeFromInProgressLog(metastoreEvents.get(2).getEventId());
|
||||
assertEquals(1, eventExecutorService.getInProgressLog().size());
|
||||
assertEquals(1, eventExecutorService.getProcessedLog().size());
|
||||
// Since 3rd event is followed by delimiter event(4th event), both events are
|
||||
// considered as processed and 5th event is already processed. So prune process log
|
||||
// discards 3rd and 4th event and retains just 5th event. So Greatest synced event id
|
||||
// should become the 5th event's id.
|
||||
lastSyncedEventId = metastoreEvents.get(4).getEventId();
|
||||
assertEquals(lastSyncedEventId, eventExecutorService.getGreatestSyncedEventId());
|
||||
assertEquals(1, eventExecutorService.getPendingEventCount(lastEventId));
|
||||
|
||||
// Remove the 6th event
|
||||
eventExecutorService.removeFromInProgressLog(lastEventId);
|
||||
assertEquals(0, eventExecutorService.getInProgressLog().size());
|
||||
assertEquals(1, eventExecutorService.getProcessedLog().size());
|
||||
// Greatest synced event id should become the 6th event's id since all the events are
|
||||
// processed
|
||||
assertEquals(lastEventId, eventExecutorService.getGreatestSyncedEventId());
|
||||
assertEquals(0, eventExecutorService.getPendingEventCount(lastEventId));
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -2058,6 +2058,7 @@ public class MetastoreEventsProcessorTest {
|
||||
assertNotNull(summaryResponse);
|
||||
// Last synced id must be set even when event processor is not active.
|
||||
assertTrue(response.isSetLast_synced_event_id());
|
||||
assertTrue(response.isSetGreatest_synced_event_id());
|
||||
} finally {
|
||||
// reset the state of event process once the test completes
|
||||
eventsProcessor_.start();
|
||||
|
||||
@@ -70,6 +70,12 @@ public class SynchronousHMSEventProcessorForTests extends MetastoreEventsProcess
|
||||
MetastoreEventsProcessor.LAST_SYNCED_EVENT_TIME).getValue();
|
||||
long latestEventTime = (Long) metrics.getGauge(
|
||||
MetastoreEventsProcessor.LATEST_EVENT_TIME).getValue();
|
||||
long greatestSyncedEventId = (Long) metrics.getGauge(
|
||||
MetastoreEventsProcessor.GREATEST_SYNCED_EVENT_ID).getValue();
|
||||
long greatestSyncedEventTime = (Long) metrics.getGauge(
|
||||
MetastoreEventsProcessor.GREATEST_SYNCED_EVENT_TIME).getValue();
|
||||
Assert.assertEquals(greatestSyncedEventId, lastSyncedEventId);
|
||||
Assert.assertEquals(greatestSyncedEventTime, lastSyncedEventTime);
|
||||
if (lastSyncedEventId == latestEventId) {
|
||||
Assert.assertEquals("Incorrect last synced event time for event " + latestEventId,
|
||||
latestEventTime, lastSyncedEventTime);
|
||||
|
||||
@@ -57,10 +57,9 @@ class EventProcessorUtils(object):
|
||||
while True:
|
||||
t = time.time()
|
||||
current_synced_id = EventProcessorUtils.get_last_synced_event_id()
|
||||
outstanding_events = EventProcessorUtils.get_outstanding_event_count()
|
||||
if current_synced_id >= target_event_id and outstanding_events == 0:
|
||||
LOG.debug("Metric last-synced-event-id has reached the desired value: %d, "
|
||||
+ "outstanding events: %d", target_event_id, outstanding_events)
|
||||
if current_synced_id >= target_event_id:
|
||||
LOG.debug("Metric last-synced-event-id has reached the desired value: %d",
|
||||
target_event_id)
|
||||
break
|
||||
status = EventProcessorUtils.get_event_processor_status()
|
||||
if status not in status_list:
|
||||
@@ -79,8 +78,7 @@ class EventProcessorUtils(object):
|
||||
if made_progress:
|
||||
LOG.debug(
|
||||
"Metric last-synced-event-id has been increased to %d but has not yet "
|
||||
+ "reached the desired value: %d, outstanding events: %d", current_synced_id,
|
||||
target_event_id, outstanding_events)
|
||||
+ "reached the desired value: %d", current_synced_id, target_event_id)
|
||||
last_synced_id = current_synced_id
|
||||
last_synced_time = t
|
||||
time.sleep(0.1)
|
||||
@@ -153,10 +151,15 @@ class EventProcessorUtils(object):
|
||||
|
||||
@staticmethod
|
||||
def get_last_synced_event_id():
|
||||
"""Returns the last_synced_event_id."""
|
||||
"""Returns the greatest event id such that all the events with id less than or
|
||||
equal to this event id are definitely synced. When hierarchical event processing is
|
||||
enabled, greatest-synced-event-id returns the value. When hierarchical event
|
||||
processing is disabled, greatest-synced-event-id and last-synced-event-id have same
|
||||
value. So we can always return greatest-synced-event-id irrespective of hierarchical
|
||||
event processing flag."""
|
||||
metrics = EventProcessorUtils.get_event_processor_metrics()
|
||||
assert 'last-synced-event-id' in metrics.keys()
|
||||
return int(metrics['last-synced-event-id'])
|
||||
assert 'greatest-synced-event-id' in metrics.keys()
|
||||
return int(metrics['greatest-synced-event-id'])
|
||||
|
||||
@staticmethod
|
||||
def get_num_skipped_events():
|
||||
@@ -174,13 +177,6 @@ class EventProcessorUtils(object):
|
||||
assert 'status' in metrics.keys()
|
||||
return metrics['status']
|
||||
|
||||
@staticmethod
|
||||
def get_outstanding_event_count():
|
||||
"""Returns the outstanding event count"""
|
||||
metrics = EventProcessorUtils.get_event_processor_metrics()
|
||||
assert 'outstanding-event-count' in metrics.keys()
|
||||
return int(metrics['outstanding-event-count'])
|
||||
|
||||
@staticmethod
|
||||
def get_current_notification_id(hive_client):
|
||||
"""Returns the current notification from metastore"""
|
||||
|
||||
Reference in New Issue
Block a user