From fe61fdf8a0962e632f176daf75b35704e5869cf3 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Tue, 27 Dec 2022 15:44:14 +0800 Subject: [PATCH 01/25] hive event init --- .../java/org/apache/doris/common/Config.java | 12 + .../doris/datasource/HMSExternalCatalog.java | 145 +++++++----- .../datasource/PooledHiveMetaStoreClient.java | 27 +++ .../hive/event/AddPartitionEvent.java | 86 +++++++ .../hive/event/AlterPartitionEvent.java | 97 ++++++++ .../hive/event/AlterTableEvent.java | 125 ++++++++++ .../datasource/hive/event/BatchEvent.java | 158 +++++++++++++ .../hive/event/CreateTableEvent.java | 60 +++++ .../hive/event/DropPartitionEvent.java | 103 +++++++++ .../datasource/hive/event/DropTableEvent.java | 81 +++++++ .../datasource/hive/event/EventFactory.java | 37 +++ .../datasource/hive/event/IgnoredEvent.java | 43 ++++ .../datasource/hive/event/InsertEvent.java | 104 +++++++++ .../datasource/hive/event/MetastoreEvent.java | 218 ++++++++++++++++++ .../hive/event/MetastoreEventFactory.java | 110 +++++++++ .../hive/event/MetastoreEventType.java | 68 ++++++ .../hive/event/MetastoreEventsProcessor.java | 151 ++++++++++++ .../event/MetastoreNotificationException.java | 37 +++ .../MetastoreNotificationFetchException.java | 37 +++ .../hive/event/MetastoreTableEvent.java | 68 ++++++ 20 files changed, 1713 insertions(+), 54 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/AddPartitionEvent.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/AlterPartitionEvent.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/AlterTableEvent.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/BatchEvent.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/CreateTableEvent.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/DropPartitionEvent.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/DropTableEvent.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/EventFactory.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/IgnoredEvent.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/InsertEvent.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEvent.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventFactory.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventType.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreNotificationException.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreNotificationFetchException.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreTableEvent.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/Config.java b/fe/fe-core/src/main/java/org/apache/doris/common/Config.java index 2bc30113e6347a..e3edff3f5d1fbd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/Config.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/Config.java @@ -1935,5 +1935,17 @@ public class Config extends ConfigBase { */ @ConfField(mutable = true) public static boolean enable_func_pushdown = true; + + /** + * Maximum number of events to poll in each RPC. + */ + @ConfField(mutable = true) + public static int hms_events_batch_size_per_rpc = 500; + + /** + * HMS polling interval in milliseconds. + */ + @ConfField + public static int hms_events_polling_interval_ms = 5000; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java index e2a5f2f3a80712..ab642ae2c38d6f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java @@ -25,14 +25,19 @@ import org.apache.doris.catalog.HiveMetaStoreClientHelper; import org.apache.doris.catalog.external.ExternalDatabase; import org.apache.doris.catalog.external.HMSExternalDatabase; +import org.apache.doris.common.Config; import org.apache.doris.common.DdlException; +import org.apache.doris.datasource.hive.event.MetastoreNotificationFetchException; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.NotificationEventResponse; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import java.io.IOException; import java.util.List; @@ -42,8 +47,12 @@ * External catalog for hive metastore compatible data sources. */ public class HMSExternalCatalog extends ExternalCatalog { + private static final Logger LOG = LogManager.getLogger(HMSExternalCatalog.class); + private static final int MAX_CLIENT_POOL_SIZE = 8; protected PooledHiveMetaStoreClient client; + // Record the latest synced event id when processing hive events + private long lastSyncedEventId = -1; /** * Default constructor for HMSExternalCatalog. @@ -57,6 +66,10 @@ public HMSExternalCatalog( props.putAll(HMSResource.getPropertiesFromDLF()); } catalogProperty = new CatalogProperty(resource, props); + // TODO: 2022/12/27 判断是否开启同步 + // if (enableHmsEventsIncrementalSync) { + setLastSyncedEventId(getCurrentEventId()); + // } } public String getHiveMetastoreUris() { @@ -64,39 +77,32 @@ public String getHiveMetastoreUris() { } @Override - protected void init() { - Map tmpDbNameToId = Maps.newConcurrentMap(); - Map tmpIdToDb = Maps.newConcurrentMap(); - InitCatalogLog initCatalogLog = new InitCatalogLog(); - initCatalogLog.setCatalogId(id); - initCatalogLog.setType(InitCatalogLog.Type.HMS); - List allDatabases = client.getAllDatabases(); - // Update the db name to id map. - for (String dbName : allDatabases) { - long dbId; - if (dbNameToId != null && dbNameToId.containsKey(dbName)) { - dbId = dbNameToId.get(dbName); - tmpDbNameToId.put(dbName, dbId); - ExternalDatabase db = idToDb.get(dbId); - db.setUnInitialized(invalidCacheInInit); - tmpIdToDb.put(dbId, db); - initCatalogLog.addRefreshDb(dbId); - } else { - dbId = Env.getCurrentEnv().getNextId(); - tmpDbNameToId.put(dbName, dbId); - HMSExternalDatabase db = new HMSExternalDatabase(this, dbId, dbName); - tmpIdToDb.put(dbId, db); - initCatalogLog.addCreateDb(dbId, dbName); - } + public void notifyPropertiesUpdated() { + initLocalObjectsImpl(); + } + + @Override + public List listDatabaseNames(SessionContext ctx) { + makeSureInitialized(); + return Lists.newArrayList(dbNameToId.keySet()); + } + + @Override + public List listTableNames(SessionContext ctx, String dbName) { + makeSureInitialized(); + HMSExternalDatabase hmsExternalDatabase = (HMSExternalDatabase) idToDb.get(dbNameToId.get(dbName)); + if (hmsExternalDatabase != null && hmsExternalDatabase.isInitialized()) { + List names = Lists.newArrayList(); + hmsExternalDatabase.getTables().forEach(table -> names.add(table.getName())); + return names; + } else { + return client.getAllTables(getRealTableName(dbName)); } - dbNameToId = tmpDbNameToId; - idToDb = tmpIdToDb; - Env.getCurrentEnv().getEditLog().logInitCatalog(initCatalogLog); } @Override - public void notifyPropertiesUpdated() { - initLocalObjectsImpl(); + public boolean tableExist(SessionContext ctx, String dbName, String tblName) { + return client.tableExists(getRealTableName(dbName), tblName); } @Override @@ -130,32 +136,34 @@ protected void initLocalObjectsImpl() { } @Override - public List listDatabaseNames(SessionContext ctx) { - makeSureInitialized(); - return Lists.newArrayList(dbNameToId.keySet()); - } - - @Override - public List listTableNames(SessionContext ctx, String dbName) { - makeSureInitialized(); - HMSExternalDatabase hmsExternalDatabase = (HMSExternalDatabase) idToDb.get(dbNameToId.get(dbName)); - if (hmsExternalDatabase != null && hmsExternalDatabase.isInitialized()) { - List names = Lists.newArrayList(); - hmsExternalDatabase.getTables().forEach(table -> names.add(table.getName())); - return names; - } else { - return client.getAllTables(getRealTableName(dbName)); + protected void init() { + Map tmpDbNameToId = Maps.newConcurrentMap(); + Map tmpIdToDb = Maps.newConcurrentMap(); + InitCatalogLog initCatalogLog = new InitCatalogLog(); + initCatalogLog.setCatalogId(id); + initCatalogLog.setType(InitCatalogLog.Type.HMS); + List allDatabases = client.getAllDatabases(); + // Update the db name to id map. + for (String dbName : allDatabases) { + long dbId; + if (dbNameToId != null && dbNameToId.containsKey(dbName)) { + dbId = dbNameToId.get(dbName); + tmpDbNameToId.put(dbName, dbId); + ExternalDatabase db = idToDb.get(dbId); + db.setUnInitialized(invalidCacheInInit); + tmpIdToDb.put(dbId, db); + initCatalogLog.addRefreshDb(dbId); + } else { + dbId = Env.getCurrentEnv().getNextId(); + tmpDbNameToId.put(dbName, dbId); + HMSExternalDatabase db = new HMSExternalDatabase(this, dbId, dbName); + tmpIdToDb.put(dbId, db); + initCatalogLog.addCreateDb(dbId, dbName); + } } - } - - @Override - public boolean tableExist(SessionContext ctx, String dbName, String tblName) { - return client.tableExists(getRealTableName(dbName), tblName); - } - - public PooledHiveMetaStoreClient getClient() { - makeSureInitialized(); - return client; + dbNameToId = tmpDbNameToId; + idToDb = tmpIdToDb; + Env.getCurrentEnv().getEditLog().logInitCatalog(initCatalogLog); } @Override @@ -170,4 +178,33 @@ public List getSchema(String dbName, String tblName) { } return tmpSchema; } + + public PooledHiveMetaStoreClient getClient() { + makeSureInitialized(); + return client; + } + + public void setLastSyncedEventId(long lastSyncedEventId) { + this.lastSyncedEventId = lastSyncedEventId; + } + + public NotificationEventResponse getNextEventResponse(String catalogName) + throws MetastoreNotificationFetchException { + if (lastSyncedEventId == -1) { + lastSyncedEventId = getCurrentEventId(); + LOG.error("Last synced event id is null when pulling events on catalog [{}]", catalogName); + return null; + } + + long currentEventId = getCurrentEventId(); + if (currentEventId == lastSyncedEventId) { + LOG.info("Event id not updated when pulling events on catalog [{}]", catalogName); + return null; + } + return client.getNextNotification(lastSyncedEventId, Config.hms_events_batch_size_per_rpc, null); + } + + public long getCurrentEventId() { + return client.getCurrentNotificationEventId().getEventId(); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/PooledHiveMetaStoreClient.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/PooledHiveMetaStoreClient.java index 05e3d9d15c666c..ce248ed7ccefae 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/PooledHiveMetaStoreClient.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/PooledHiveMetaStoreClient.java @@ -19,6 +19,7 @@ import org.apache.doris.catalog.HMSResource; import org.apache.doris.common.Config; +import org.apache.doris.datasource.hive.event.MetastoreNotificationFetchException; import com.aliyun.datalake.metastore.hive2.ProxyMetaStoreClient; import com.google.common.base.Preconditions; @@ -29,8 +30,10 @@ import org.apache.hadoop.hive.metastore.IMetaStoreClient; import org.apache.hadoop.hive.metastore.RetryingMetaStoreClient; import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj; +import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId; import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.metastore.api.NotificationEventResponse; import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.logging.log4j.LogManager; @@ -145,6 +148,30 @@ public Map> getPartitionColumnStatistics( } } + public CurrentNotificationEventId getCurrentNotificationEventId() { + try (CachedClient client = getClient()) { + return client.client.getCurrentNotificationEventId(); + } catch (Exception e) { + LOG.error("Failed to fetch current notification event id", e); + throw new MetastoreNotificationFetchException( + "Failed to get current notification event id. msg: " + e.getMessage()); + } + } + + public NotificationEventResponse getNextNotification(long lastEventId, + int maxEvents, + IMetaStoreClient.NotificationFilter filter) + throws MetastoreNotificationFetchException { + try (CachedClient client = getClient()) { + return client.client.getNextNotification(lastEventId, maxEvents, filter); + } catch (Exception e) { + LOG.error("Failed to get next notification based on last event id {}", lastEventId, e); + throw new MetastoreNotificationFetchException( + "Failed to get next notification based on last event id: " + lastEventId + ". msg: " + e + .getMessage()); + } + } + private class CachedClient implements AutoCloseable { private final IMetaStoreClient client; diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/AddPartitionEvent.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/AddPartitionEvent.java new file mode 100644 index 00000000000000..2f69cce2558889 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/AddPartitionEvent.java @@ -0,0 +1,86 @@ +// 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.doris.datasource.hive.event; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import org.apache.hadoop.hive.metastore.api.NotificationEvent; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.messaging.AddPartitionMessage; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.List; + +/** + * MetastoreEvent for ADD_PARTITION event type + */ +public class AddPartitionEvent extends MetastoreTableEvent { + private static final Logger LOG = LogManager.getLogger(AddPartitionEvent.class); + + private final Partition addedPartition; + + /** + * Prevent instantiation from outside should use MetastoreEventFactory instead + */ + private AddPartitionEvent(NotificationEvent event, + Partition addedPartition, + String catalogName) { + super(event, catalogName); + Preconditions.checkState(getEventType().equals(MetastoreEventType.ADD_PARTITION)); + if (event.getMessage() == null) { + throw new IllegalStateException(debugString("Event message is null")); + } + + try { + AddPartitionMessage addPartitionMessage = + MetastoreEventsProcessor.getMessageDeserializer() + .getAddPartitionMessage(event.getMessage()); + this.addedPartition = addedPartition; + hmsTbl = addPartitionMessage.getTableObj(); + // hivePartitionNames.clear(); + // List partitionColNames = hmsTbl.getPartitionKeys().stream() + // .map(FieldSchema::getName).collect(Collectors.toList()); + // hivePartitionNames.add(HivePartitionName.of(dbName, tblName, + // FileUtils.makePartName(partitionColNames, addedPartition.getValues()))); + } catch (Exception ex) { + throw new MetastoreNotificationException(ex); + } + } + + protected static List getEvents(NotificationEvent event, + String catalogName) { + List addPartitionEvents = Lists.newArrayList(); + try { + AddPartitionMessage addPartitionMessage = + MetastoreEventsProcessor.getMessageDeserializer() + .getAddPartitionMessage(event.getMessage()); + addPartitionMessage.getPartitionObjs().forEach(partition -> + addPartitionEvents.add(new AddPartitionEvent(event, partition, catalogName))); + } catch (Exception ex) { + throw new MetastoreNotificationException(ex); + } + return addPartitionEvents; + } + + @Override + protected void process() throws MetastoreNotificationException { + throw new UnsupportedOperationException("Unsupported event type: " + getEventType()); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/AlterPartitionEvent.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/AlterPartitionEvent.java new file mode 100644 index 00000000000000..b038b7a834fdd2 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/AlterPartitionEvent.java @@ -0,0 +1,97 @@ +// 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.doris.datasource.hive.event; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import org.apache.hadoop.hive.metastore.api.NotificationEvent; +import org.apache.hadoop.hive.metastore.api.Partition; +import org.apache.hadoop.hive.metastore.messaging.AlterPartitionMessage; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.List; + +/** + * MetastoreEvent for ALTER_PARTITION event type + */ +public class AlterPartitionEvent extends MetastoreTableEvent { + private static final Logger LOG = LogManager.getLogger(AlterPartitionEvent.class); + + // the Partition object before alter operation, as parsed from the NotificationEvent + private final Partition partitionBefore; + // the Partition object after alter operation, as parsed from the NotificationEvent + private final Partition partitionAfter; + + private AlterPartitionEvent(NotificationEvent event, String catalogName) { + super(event, catalogName); + Preconditions.checkState(getEventType() == MetastoreEventType.ALTER_PARTITION); + Preconditions.checkNotNull(event.getMessage()); + AlterPartitionMessage alterPartitionMessage = + MetastoreEventsProcessor.getMessageDeserializer() + .getAlterPartitionMessage(event.getMessage()); + + try { + partitionBefore = Preconditions.checkNotNull(alterPartitionMessage.getPtnObjBefore()); + partitionAfter = Preconditions.checkNotNull(alterPartitionMessage.getPtnObjAfter()); + hmsTbl = alterPartitionMessage.getTableObj(); + } catch (Exception e) { + throw new MetastoreNotificationException( + debugString("Unable to parse the alter partition message"), e); + } + } + + public static List getEvents(NotificationEvent event, + String catalogName) { + return Lists.newArrayList(new AlterPartitionEvent(event, catalogName)); + } + + @Override + protected boolean canBeBatched(MetastoreEvent event) { + return true; + } + + @Override + protected MetastoreEvent addToBatchEvents(MetastoreEvent event) { + BatchEvent batchEvent = new BatchEvent<>(this); + Preconditions.checkState(batchEvent.canBeBatched(event)); + batchEvent.addToBatchEvents(event); + return batchEvent; + } + + @Override + protected boolean existInCache() { + return true; + } + + @Override + protected boolean canBeSkipped() { + return false; + } + + @Override + protected boolean isSupported() { + return true; + } + + @Override + protected void process() throws MetastoreNotificationException { + + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/AlterTableEvent.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/AlterTableEvent.java new file mode 100644 index 00000000000000..f3ab07d011da8a --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/AlterTableEvent.java @@ -0,0 +1,125 @@ +// 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.doris.datasource.hive.event; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.NotificationEvent; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.metastore.messaging.json.JSONAlterTableMessage; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.List; + +/** + * MetastoreEvent for ALTER_TABLE event type + */ +public class AlterTableEvent extends MetastoreTableEvent { + private static final Logger LOG = LogManager.getLogger(AlterTableEvent.class); + + // the table object before alter operation, as parsed from the NotificationEvent + protected Table tableBefore; + // the table object after alter operation, as parsed from the NotificationEvent + protected Table tableAfter; + // true if this alter event was due to a rename operation + protected final boolean isRename; + // true if this alter event was due to a schema change operation + protected boolean isSchemaChange = false; + + private AlterTableEvent(NotificationEvent event, String catalogName) { + super(event, catalogName); + Preconditions.checkArgument(MetastoreEventType.ALTER_TABLE.equals(getEventType())); + JSONAlterTableMessage alterTableMessage = + (JSONAlterTableMessage) MetastoreEventsProcessor.getMessageDeserializer() + .getAlterTableMessage(event.getMessage()); + try { + hmsTbl = Preconditions.checkNotNull(alterTableMessage.getTableObjBefore()); + tableAfter = Preconditions.checkNotNull(alterTableMessage.getTableObjAfter()); + tableBefore = Preconditions.checkNotNull(alterTableMessage.getTableObjBefore()); + } catch (Exception e) { + throw new MetastoreNotificationException( + debugString("Unable to parse the alter table message"), e); + } + // this is a rename event if either dbName or tblName of before and after object changed + isRename = !hmsTbl.getDbName().equalsIgnoreCase(tableAfter.getDbName()) + || !hmsTbl.getTableName().equalsIgnoreCase(tableAfter.getTableName()); + } + + public static List getEvents(NotificationEvent event, + String catalogName) { + return Lists.newArrayList(new AlterTableEvent(event, catalogName)); + } + + private boolean isSchemaChange(List before, List after) { + if (before.size() != after.size()) { + return true; + } + + if (!before.equals(after)) { + return true; + } + + return false; + } + + public boolean isSchemaChange() { + return isSchemaChange; + } + + @Override + public boolean canBeBatched(MetastoreEvent event) { + return true; + } + + @Override + protected MetastoreEvent addToBatchEvents(MetastoreEvent event) { + BatchEvent batchEvent = new BatchEvent<>(this); + Preconditions.checkState(batchEvent.canBeBatched(event)); + batchEvent.addToBatchEvents(event); + return batchEvent; + } + + @Override + protected boolean existInCache() { + + return true; + + } + + @Override + protected boolean canBeSkipped() { + return false; + } + + @Override + protected boolean isSupported() { + return true; + } + + public boolean isRename() { + return isRename; + } + + @Override + protected void process() throws MetastoreNotificationException { + + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/BatchEvent.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/BatchEvent.java new file mode 100644 index 00000000000000..7c84aa6bad1b58 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/BatchEvent.java @@ -0,0 +1,158 @@ +// 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.doris.datasource.hive.event; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.List; +import java.util.stream.Collectors; + +/** + * This event represents a batch of events of type T. The batch of events is + * initialized from a single initial event called baseEvent. More events can be added + * to the batch using {@code addToBatchEvents} method. + * + * @param The type of event which is batched by this event. + */ +public class BatchEvent extends MetastoreTableEvent { + private static final Logger LOG = LogManager.getLogger(BatchEvent.class); + + private final T baseEvent; + private final List batchedEvents = Lists.newArrayList(); + + protected BatchEvent(T baseEvent) { + super(baseEvent.event, baseEvent.catalogName); + this.hmsTbl = baseEvent.hmsTbl; + this.baseEvent = baseEvent; + batchedEvents.add(baseEvent); + } + + /** + * @param event The event under consideration to be batched into this event. It can + * be added to the batch if it can be batched into the last event of the current batch. + * @return true if we can add the event to the current batch; else false. + */ + @Override + protected boolean canBeBatched(MetastoreEvent event) { + Preconditions.checkState(!batchedEvents.isEmpty()); + return batchedEvents.get(batchedEvents.size() - 1).canBeBatched(event); + } + + @Override + public MetastoreEvent addToBatchEvents(MetastoreEvent event) { + Preconditions.checkState(canBeBatched(event)); + batchedEvents.add((T) event); + return this; + } + + /** + * Return the event id of this batch event. We return the last eventId + * from this batch which is important since it is used to determined the event + * id for fetching next set of events from metastore. + */ + @Override + public long getEventId() { + Preconditions.checkState(!batchedEvents.isEmpty()); + return batchedEvents.get(batchedEvents.size() - 1).getEventId(); + } + + @Override + public int getNumberOfEvents() { + return batchedEvents.size(); + } + + public List getBatchEvents() { + return batchedEvents; + } + + /** + * Gets the event id of the first event in the batch. + */ + public long getFirstEventId() { + return batchedEvents.get(0).getEventId(); + } + + /** + * Gets the event id of the last event in the batch. + */ + public long getLastEventId() { + return batchedEvents.get(batchedEvents.size() - 1).getEventId(); + } + + /** + * Gets the event of the last event in the batch. + */ + protected MetastoreEvent getLastEvent() { + return batchedEvents.get(batchedEvents.size() - 1); + } + + @Override + protected boolean isSupported() { + return true; + } + + @Override + protected boolean existInCache() throws MetastoreNotificationException { + return baseEvent.existInCache(); + } + + /** + * The merge rule in batch event is relatively simple, because some events that don't need to be processed + * have been filtered out during the {@link MetastoreEventFactory#createBatchEvents}. mainly the following two. + * 1. Multiple ALTER_PARTITION or ALTER_TABLE only process the last one. + * 2. The first event in the batch event is ADD_PARTITION, and the ALTER events currently only takes the last one. + */ + protected List mergeBatchEvents(List eventToMerge) { + List mergedEvents = Lists.newArrayList(); + T first = eventToMerge.get(0); + T last = eventToMerge.get(eventToMerge.size() - 1); + if (first.getEventType() == MetastoreEventType.ADD_PARTITION) { + mergedEvents.add(first); + } + mergedEvents.add(last); + return mergedEvents; + } + + @Override + protected void process() throws MetastoreNotificationException { + LOG.info("Start to process batch event for {} events from {} to {}", + getNumberOfEvents(), getFirstEventId(), getLastEvent()); + if (!baseEvent.existInCache()) { + return; + } + + List eventsToProcess = batchedEvents.stream() + .filter(event -> !event.canBeSkipped()) + .collect(Collectors.toList()); + + if (eventsToProcess.isEmpty()) { + LOG.info("Ignoring events {} since they modify parameters which can be ignored", batchedEvents); + return; + } + + if (eventsToProcess.size() > 1) { + eventsToProcess = mergeBatchEvents(eventsToProcess); + } + + eventsToProcess.forEach(MetastoreEvent::process); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/CreateTableEvent.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/CreateTableEvent.java new file mode 100644 index 00000000000000..0880ce95867c46 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/CreateTableEvent.java @@ -0,0 +1,60 @@ +// 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.doris.datasource.hive.event; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import org.apache.hadoop.hive.metastore.api.NotificationEvent; +import org.apache.hadoop.hive.metastore.messaging.CreateTableMessage; + +import java.util.List; + +/** + * MetastoreEvent for CREATE_TABLE event type + */ +public class CreateTableEvent extends MetastoreTableEvent { + public static final String CREATE_TABLE_EVENT_TYPE = "CREATE_TABLE"; + + public static List getEvents(NotificationEvent event, + String catalogName) { + return Lists.newArrayList(new CreateTableEvent(event, catalogName)); + } + + private CreateTableEvent(NotificationEvent event, + String catalogName) + throws MetastoreNotificationException { + super(event, catalogName); + Preconditions.checkArgument(MetastoreEventType.CREATE_TABLE.equals(getEventType())); + Preconditions.checkNotNull(MetastoreEventType.CREATE_TABLE, debugString("Event message is null")); + CreateTableMessage createTableMessage = + MetastoreEventsProcessor.getMessageDeserializer().getCreateTableMessage(event.getMessage()); + + try { + hmsTbl = createTableMessage.getTableObj(); + } catch (Exception e) { + throw new MetastoreNotificationException( + debugString("Unable to deserialize the event message"), e); + } + } + + @Override + protected void process() throws MetastoreNotificationException { + throw new UnsupportedOperationException("Unsupported event type: " + getEventType()); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/DropPartitionEvent.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/DropPartitionEvent.java new file mode 100644 index 00000000000000..8ada42e4440fcf --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/DropPartitionEvent.java @@ -0,0 +1,103 @@ +// 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.doris.datasource.hive.event; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import org.apache.hadoop.hive.metastore.api.NotificationEvent; +import org.apache.hadoop.hive.metastore.messaging.DropPartitionMessage; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.List; +import java.util.Map; + +/** + * MetastoreEvent for DROP_PARTITION event type + */ +public class DropPartitionEvent extends MetastoreTableEvent { + private static final Logger LOG = LogManager.getLogger(DropPartitionEvent.class); + public static final String EVENT_TYPE = "DROP_PARTITION"; + + private final Map droppedPartition; + + private DropPartitionEvent(NotificationEvent event, + Map droppedPartition, + String catalogName) { + super(event, catalogName); + Preconditions.checkState(getEventType().equals(MetastoreEventType.DROP_PARTITION)); + Preconditions.checkNotNull(event.getMessage()); + DropPartitionMessage dropPartitionMessage = + MetastoreEventsProcessor.getMessageDeserializer() + .getDropPartitionMessage(event.getMessage()); + try { + hmsTbl = Preconditions.checkNotNull(dropPartitionMessage.getTableObj()); + Preconditions.checkNotNull(droppedPartition); + this.droppedPartition = droppedPartition; + } catch (Exception ex) { + throw new MetastoreNotificationException( + debugString("Could not parse drop event message. "), ex); + } + } + + protected static List getEvents(NotificationEvent event, + String catalogName) { + DropPartitionMessage dropPartitionMessage = + MetastoreEventsProcessor.getMessageDeserializer() + .getDropPartitionMessage(event.getMessage()); + List dropPartitionEvents = Lists.newArrayList(); + try { + List> droppedPartitions = dropPartitionMessage.getPartitions(); + droppedPartitions.forEach(part -> + dropPartitionEvents.add(new DropPartitionEvent(event, part, catalogName))); + } catch (Exception e) { + throw new MetastoreNotificationException(e); + } + return dropPartitionEvents; + } + + @Override + public boolean canBeBatched(MetastoreEvent event) { + return true; + } + + @Override + protected MetastoreEvent addToBatchEvents(MetastoreEvent event) { + BatchEvent batchEvent = new BatchEvent<>(this); + Preconditions.checkState(batchEvent.canBeBatched(event)); + batchEvent.addToBatchEvents(event); + return batchEvent; + } + + @Override + protected boolean existInCache() { + // return cache.isPartitionPresent(getHivePartitionName()); + return true; + } + + @Override + protected boolean isSupported() { + return true; + } + + @Override + protected void process() throws MetastoreNotificationException { + + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/DropTableEvent.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/DropTableEvent.java new file mode 100644 index 00000000000000..ff4acec1cecbc2 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/DropTableEvent.java @@ -0,0 +1,81 @@ +// 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.doris.datasource.hive.event; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import org.apache.hadoop.hive.metastore.api.NotificationEvent; +import org.apache.hadoop.hive.metastore.messaging.json.JSONDropTableMessage; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.List; + +/** + * MetastoreEvent for DROP_TABLE event type + */ +public class DropTableEvent extends MetastoreTableEvent { + private static final Logger LOG = LogManager.getLogger(DropTableEvent.class); + private final String dbName; + private final String tableName; + + private DropTableEvent(NotificationEvent event, + String catalogName) { + super(event, catalogName); + Preconditions.checkArgument(MetastoreEventType.DROP_TABLE.equals(getEventType())); + JSONDropTableMessage dropTableMessage = + (JSONDropTableMessage) MetastoreEventsProcessor.getMessageDeserializer() + .getDropTableMessage(event.getMessage()); + try { + dbName = dropTableMessage.getDB(); + tableName = dropTableMessage.getTable(); + } catch (Exception e) { + throw new MetastoreNotificationException(debugString( + "Could not parse event message. " + + "Check if %s is set to true in metastore configuration", + MetastoreEventsProcessor.HMS_ADD_THRIFT_OBJECTS_IN_EVENTS_CONFIG_KEY), e); + } + } + + public static List getEvents(NotificationEvent event, + String catalogName) { + return Lists.newArrayList(new DropTableEvent(event, catalogName)); + } + + @Override + protected boolean existInCache() { + // return cache.isTablePresent(HiveTableName.of(dbName, tableName)); + return true; + } + + @Override + protected boolean canBeSkipped() { + return false; + } + + protected boolean isSupported() { + // return !isResourceMappingCatalog(catalogName); + return true; + } + + @Override + protected void process() throws MetastoreNotificationException { + + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/EventFactory.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/EventFactory.java new file mode 100644 index 00000000000000..6ea2341171cde4 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/EventFactory.java @@ -0,0 +1,37 @@ +// 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.doris.datasource.hive.event; + +import org.apache.hadoop.hive.metastore.api.NotificationEvent; + +import java.util.List; + +/** + * Factory interface to generate a {@link MetastoreEvent} from a {@link NotificationEvent} object. + */ +public interface EventFactory { + + + // List get(NotificationEvent hmsEvent, + // CacheUpdateProcessor cacheProcessor, + // String catalogName) throws MetastoreNotificationException; + + List get(NotificationEvent hmsEvent, + String catalogName) throws MetastoreNotificationException; +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/IgnoredEvent.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/IgnoredEvent.java new file mode 100644 index 00000000000000..4d2dc1a178d31c --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/IgnoredEvent.java @@ -0,0 +1,43 @@ +// 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.doris.datasource.hive.event; + +import com.google.common.collect.Lists; +import org.apache.hadoop.hive.metastore.api.NotificationEvent; + +import java.util.List; + +/** + * An event type which is ignored. Useful for unsupported metastore event types + */ +public class IgnoredEvent extends MetastoreEvent { + protected IgnoredEvent(NotificationEvent event, String catalogName) { + super(event, catalogName); + } + + private static List getEvents(NotificationEvent event, + String catalogName) { + return Lists.newArrayList(new IgnoredEvent(event, catalogName)); + } + + @Override + public void process() { + debugLog("Ignoring unknown event type " + metastoreNotificationEvent.getEventType()); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/InsertEvent.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/InsertEvent.java new file mode 100644 index 00000000000000..1403ac39ea321f --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/InsertEvent.java @@ -0,0 +1,104 @@ +// 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.doris.datasource.hive.event; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import org.apache.hadoop.hive.metastore.api.NotificationEvent; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.Collections; +import java.util.List; + +/** + * Metastore event handler for INSERT events. Handles insert events at both table and partition scopes. + * If partition is null, treat it as ALTER_TABLE event, otherwise as ALTER_PARTITION event. + */ +public class InsertEvent extends MetastoreTableEvent { + private static final Logger LOG = LogManager.getLogger(InsertEvent.class); + + // Represents the partition for this insert. Null if the table is unpartitioned. + // private final Partition insertPartition; + + private InsertEvent(NotificationEvent event, String catalogName) { + super(event, catalogName); + Preconditions.checkArgument(MetastoreEventType.INSERT.equals(getEventType())); + try { + // hmsTbl = Preconditions.checkNotNull(insertMessage.getTableObj()); + // insertPartition = insertMessage.getPtnObj(); + // if (insertPartition != null) { + // List partitionColNames = hmsTbl.getPartitionKeys().stream() + // .map(FieldSchema::getName).collect(Collectors.toList()); + // hivePartitionNames.add(HivePartitionName.of(dbName, tblName, + // FileUtils.makePartName(partitionColNames, insertPartition.getValues()))); + // } + } catch (Exception e) { + LOG.warn( + "The InsertEvent of the current hive version cannot be parsed,and there will be a corresponding " + + "Alter Event in next, InsertEvent is ignored here. {}", + e.getMessage()); + throw new MetastoreNotificationException(debugString("Unable to " + + "parse insert message"), e); + } + } + + public static List getEvents(NotificationEvent event, + String catalogName) { + try { + return Lists.newArrayList(new InsertEvent(event, catalogName)); + } catch (MetastoreNotificationException e) { + return Collections.emptyList(); + } + } + + @Override + public boolean canBeBatched(MetastoreEvent event) { + return true; + } + + @Override + protected MetastoreEvent addToBatchEvents(MetastoreEvent event) { + BatchEvent batchEvent = new BatchEvent<>(this); + Preconditions.checkState(batchEvent.canBeBatched(event)); + batchEvent.addToBatchEvents(event); + return batchEvent; + } + + @Override + protected boolean existInCache() { + return true; + } + + @Override + protected boolean isSupported() { + return true; + } + + public boolean isPartitionTbl() { + return true; + } + + @Override + protected void process() throws MetastoreNotificationException { + if (!existInCache()) { + return; + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEvent.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEvent.java new file mode 100644 index 00000000000000..96dd3d0a244713 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEvent.java @@ -0,0 +1,218 @@ +// 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.doris.datasource.hive.event; + +import org.apache.hadoop.hive.metastore.api.NotificationEvent; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +/** + * Abstract base class for all MetastoreEvents. A MetastoreEvent is an object used to + * process a NotificationEvent received from metastore. + */ +public abstract class MetastoreEvent { + private static final Logger LOG = LogManager.getLogger(MetastoreEvent.class); + // String.format compatible string to prepend event id and type + private static final String STR_FORMAT_EVENT_ID_TYPE = "EventId: %d EventType: %s "; + + // logger format compatible string to prepend to a log formatted message + private static final String LOG_FORMAT_EVENT_ID_TYPE = "EventId: {} EventType: {} "; + + // the notification received from metastore which is processed by this + protected final NotificationEvent event; + + // dbName from the event + protected final String dbName; + + // tblName from the event + protected final String tblName; + + // eventId of the event. Used instead of calling getter on event everytime + private final long eventId; + + // eventType from the NotificationEvent + private final MetastoreEventType eventType; + + // Actual notificationEvent object received from Metastore + protected final NotificationEvent metastoreNotificationEvent; + + // The cached instance of this event that needs to be updated + // TODO: 2022/12/27 注释 + // protected final CacheUpdateProcessor cache; + + protected final String catalogName; + + // protected MetastoreEvent(NotificationEvent event, CacheUpdateProcessor cacheProcessor, String catalogName) { + // this.event = event; + // this.dbName = event.getDbName(); + // this.tblName = event.getTableName(); + // this.eventId = event.getEventId(); + // this.eventType = MetastoreEventType.from(event.getEventType()); + // this.metastoreNotificationEvent = event; + // this.cache = cacheProcessor; + // this.catalogName = catalogName; + // } + + protected MetastoreEvent(NotificationEvent event, String catalogName) { + this.event = event; + this.dbName = event.getDbName(); + this.tblName = event.getTableName(); + this.eventId = event.getEventId(); + this.eventType = MetastoreEventType.from(event.getEventType()); + this.metastoreNotificationEvent = event; + this.catalogName = catalogName; + } + + public long getEventId() { + return eventId; + } + + public MetastoreEventType getEventType() { + return eventType; + } + + public String getDbName() { + return dbName; + } + + public String getTblName() { + return tblName; + } + + /** + * Checks if the given event can be batched into this event. Default behavior is + * to return false which can be overridden by a sub-class. + * The current version is relatively simple to process batch events, so all that need to be processed are true. + * + * @param event The event under consideration to be batched into this event. + * @return false if event cannot be batched into this event; otherwise true. + */ + protected boolean canBeBatched(MetastoreEvent event) { + return false; + } + + /** + * Adds the given event into the batch of events represented by this event. Default + * implementation is to return null. Sub-classes must override this method to + * implement batching. + * + * @param event The event which needs to be added to the batch. + * @return The batch event which represents all the events batched into this event + * until now including the given event. + */ + protected MetastoreEvent addToBatchEvents(MetastoreEvent event) { + return null; + } + + + protected boolean existInCache() throws MetastoreNotificationException { + return false; + } + + /** + * Returns the number of events represented by this event. For most events this is 1. + * In case of batch events this could be more than 1. + */ + protected int getNumberOfEvents() { + return 1; + } + + /** + * Certain events like ALTER_TABLE or ALTER_PARTITION implement logic to ignore + * some events because they do not affect query results. + * + * @return true if this event can be skipped. + */ + protected boolean canBeSkipped() { + return false; + } + + /** + * Whether the current version of FE supports processing of some events, some events are reserved, + * and may be processed later version, such as {@link CreateTableEvent} + */ + protected boolean isSupported() { + return false; + } + + /** + * Process the information available in the NotificationEvent. + */ + protected abstract void process() throws MetastoreNotificationException; + + /** + * Helper method to get debug string with helpful event information prepended to the + * message. This can be used to generate helpful exception messages + * + * @param msgFormatString String value to be used in String.format() for the given message + * @param args args to the String.format() for the given msgFormatString + */ + protected String debugString(String msgFormatString, Object... args) { + String formatString = STR_FORMAT_EVENT_ID_TYPE + msgFormatString; + Object[] formatArgs = getLogFormatArgs(args); + return String.format(formatString, formatArgs); + } + + /** + * Helper method to generate the format args after prepending the event id and type + */ + private Object[] getLogFormatArgs(Object[] args) { + Object[] formatArgs = new Object[args.length + 2]; + formatArgs[0] = getEventId(); + formatArgs[1] = getEventType(); + int i = 2; + for (Object arg : args) { + formatArgs[i] = arg; + i++; + } + return formatArgs; + } + + /** + * Logs at info level the given log formatted string and its args. The log formatted + * string should have {} pair at the appropriate location in the string for each arg + * value provided. This method prepends the event id and event type before logging the + * message. No-op if the log level is not at INFO + */ + protected void infoLog(String logFormattedStr, Object... args) { + if (!LOG.isInfoEnabled()) { + return; + } + String formatString = LOG_FORMAT_EVENT_ID_TYPE + logFormattedStr; + Object[] formatArgs = getLogFormatArgs(args); + LOG.info(formatString, formatArgs); + } + + /** + * Similar to infoLog excepts logs at debug level + */ + protected void debugLog(String logFormattedStr, Object... args) { + if (!LOG.isDebugEnabled()) { + return; + } + String formatString = LOG_FORMAT_EVENT_ID_TYPE + logFormattedStr; + Object[] formatArgs = getLogFormatArgs(args); + LOG.debug(formatString, formatArgs); + } + + @Override + public String toString() { + return String.format(STR_FORMAT_EVENT_ID_TYPE, eventId, eventType); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventFactory.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventFactory.java new file mode 100644 index 00000000000000..0400a77349c4a3 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventFactory.java @@ -0,0 +1,110 @@ +// 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.doris.datasource.hive.event; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import org.apache.hadoop.hive.metastore.api.NotificationEvent; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +/** + * Factory class to create various MetastoreEvents. + */ +public class MetastoreEventFactory implements EventFactory { + private static final Logger LOG = LogManager.getLogger(MetastoreEventFactory.class); + private final List externalTables; + + public MetastoreEventFactory(List externalTables) { + this.externalTables = externalTables; + } + + public boolean needToProcess(String catalogTableName) { + return externalTables.contains(catalogTableName); + } + + /** + * For an {@link AddPartitionEvent} and {@link DropPartitionEvent} drop event, + * we need to divide it into multiple events according to the number of partitions it processes. + * It is convenient for creating batch tasks to parallel processing. + */ + @Override + public List get(NotificationEvent event, + String catalogName) { + Preconditions.checkNotNull(event.getEventType()); + MetastoreEventType metastoreEventType = MetastoreEventType.from(event.getEventType()); + switch (metastoreEventType) { + case CREATE_TABLE: + return CreateTableEvent.getEvents(event, catalogName); + case ALTER_TABLE: + return AlterTableEvent.getEvents(event, catalogName); + case DROP_TABLE: + return DropTableEvent.getEvents(event, catalogName); + case ALTER_PARTITION: + return AlterPartitionEvent.getEvents(event, catalogName); + case DROP_PARTITION: + return DropPartitionEvent.getEvents(event, catalogName); + case INSERT: + return InsertEvent.getEvents(event, catalogName); + default: + // ignore all the unknown events by creating a IgnoredEvent + return Lists.newArrayList(new IgnoredEvent(event, catalogName)); + } + } + + List getFilteredEvents(List events, + String catalogName) { + List metastoreEvents = Lists.newArrayList(); + + // Currently, the hive external table needs to be manually created in StarRocks to map with the hms table. + // Therefore, it's necessary to filter the events pulled this time from the hms instance, + // and the events of the tables that don't register in the fe MetastoreEventsProcessor need to be filtered out. + for (NotificationEvent event : events) { + // String dbName = event.getDbName(); + // String tableName = event.getTableName(); + + + metastoreEvents.addAll(get(event, catalogName)); + } + + List tobeProcessEvents = metastoreEvents.stream() + .filter(MetastoreEvent::isSupported) + .collect(Collectors.toList()); + + if (tobeProcessEvents.isEmpty()) { + LOG.warn("The metastore events to process is empty on catalog {}", catalogName); + return Collections.emptyList(); + } + + return createBatchEvents(tobeProcessEvents); + } + + /** + * Create batch event tasks according to HivePartitionName to facilitate subsequent parallel processing. + * For ADD_PARTITION and DROP_PARTITION, we directly override any events before that partition. + * For a partition, it is meaningless to process any events before the drop partition. + */ + List createBatchEvents(List events) { + return events; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventType.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventType.java new file mode 100644 index 00000000000000..31dce2936645fc --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventType.java @@ -0,0 +1,68 @@ +// 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.doris.datasource.hive.event; + +/** + * Currently we only support handling some events. + */ +public enum MetastoreEventType { + CREATE_TABLE("CREATE_TABLE"), + DROP_TABLE("DROP_TABLE"), + ALTER_TABLE("ALTER_TABLE"), + CREATE_DATABASE("CREATE_DATABASE"), + DROP_DATABASE("DROP_DATABASE"), + ALTER_DATABASE("ALTER_DATABASE"), + ADD_PARTITION("ADD_PARTITION"), + ALTER_PARTITION("ALTER_PARTITION"), + ALTER_PARTITIONS("ALTER_PARTITIONS"), + DROP_PARTITION("DROP_PARTITION"), + INSERT("INSERT"), + INSERT_PARTITIONS("INSERT_PARTITIONS"), + ALLOC_WRITE_ID_EVENT("ALLOC_WRITE_ID_EVENT"), + COMMIT_TXN("COMMIT_TXN"), + ABORT_TXN("ABORT_TXN"), + OTHER("OTHER"); + + private final String eventType; + + MetastoreEventType(String msEventType) { + this.eventType = msEventType; + } + + @Override + public String toString() { + return eventType; + } + + /** + * Returns the MetastoreEventType from a given string value of event from Metastore's + * NotificationEvent.eventType. If none of the supported MetastoreEventTypes match, + * return OTHER + * + * @param eventType EventType value from the {@link org.apache.hadoop.hive.metastore.api.NotificationEvent} + */ + public static MetastoreEventType from(String eventType) { + for (MetastoreEventType metastoreEventType : values()) { + if (metastoreEventType.eventType.equalsIgnoreCase(eventType)) { + return metastoreEventType; + } + } + return OTHER; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java new file mode 100644 index 00000000000000..ac51edc4796c8e --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java @@ -0,0 +1,151 @@ +// 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.doris.datasource.hive.event; + +import org.apache.doris.catalog.Env; +import org.apache.doris.common.Config; +import org.apache.doris.common.util.MasterDaemon; +import org.apache.doris.datasource.CatalogIf; +import org.apache.doris.datasource.HMSExternalCatalog; + +import com.google.common.collect.Lists; +import org.apache.hadoop.hive.metastore.api.NotificationEvent; +import org.apache.hadoop.hive.metastore.api.NotificationEventResponse; +import org.apache.hadoop.hive.metastore.messaging.MessageDeserializer; +import org.apache.hadoop.hive.metastore.messaging.json.JSONMessageDeserializer; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.Collections; +import java.util.List; + +/** + * A metastore event is a instance of the class + * {@link NotificationEvent}. Metastore can be + * configured, to work with Listeners which are called on various DDL operations like + * create/alter/drop operations on database, table, partition etc. Each event has a unique + * incremental id and the generated events are be fetched from Metastore to get + * incremental updates to the metadata stored in Hive metastore using the the public API + * get_next_notification These events could be generated by external + * Metastore clients like Apache Hive or Apache Spark configured to talk with the same metastore. + *

+ * This class is used to poll metastore for such events at a given frequency. By observing + * such events, we can take appropriate action on the {@link org.apache.doris.datasource.hive.HiveMetaStoreCache} + * (refresh/invalidate/add/remove) so that represents the latest information + * available in metastore. We keep track of the last synced event id in each polling + * iteration so the next batch can be requested appropriately. The current batch size is + * constant and set to {@link org.apache.doris.common.Config#hms_events_batch_size_per_rpc}. + */ +public class MetastoreEventsProcessor extends MasterDaemon { + private static final Logger LOG = LogManager.getLogger(MetastoreEventsProcessor.class); + public static final String HMS_ADD_THRIFT_OBJECTS_IN_EVENTS_CONFIG_KEY = + "hive.metastore.notifications.add.thrift.objects"; + + // for deserializing from JSON strings from metastore event + private static final MessageDeserializer MESSAGE_DESERIALIZER = new JSONMessageDeserializer(); + + + // event factory which is used to get or create MetastoreEvents + private final MetastoreEventFactory metastoreEventFactory; + + // [catalogName.dbName.tableName] for hive table with resource + private final List externalTables = Lists.newArrayList(); + + public MetastoreEventsProcessor() { + super(MetastoreEventsProcessor.class.getName(), Config.hms_events_polling_interval_ms); + this.metastoreEventFactory = new MetastoreEventFactory(externalTables); + } + + /** + * Fetch the next batch of NotificationEvents from metastore. The default batch size is + * {@link Config#hms_events_batch_size_per_rpc} + */ + private List getNextHMSEvents(HMSExternalCatalog hmsExternalCatalog) { + LOG.info("Start to pull events on catalog [{}]", hmsExternalCatalog.getName()); + NotificationEventResponse response = hmsExternalCatalog.getNextEventResponse(hmsExternalCatalog.getName()); + + if (response == null) { + return Collections.emptyList(); + } + return response.getEvents(); + } + + + // private void doExecute(List events) { + // for (MetastoreEvent event : events) { + // try { + // event.process(); + // } catch (Exception e) { + // if (event instanceof BatchEvent) { + // cacheProcessor.setLastSyncedEventId(((BatchEvent) event).getFirstEventId() - 1); + // } else { + // cacheProcessor.setLastSyncedEventId(event.getEventId() - 1); + // } + // throw e; + // } + // } + // } + + private void doExecute(List events) { + for (NotificationEvent event : events) { + LOG.info("event消息内容:{}", event.toString()); + // LOG.info("收到hive event消息,id:为{},时间为:{},db为:{},table为:{},type为:{}",event.getEventId(),); + } + } + + /** + * Process the given list of notification events. Useful for tests which provide a list of events + */ + private void processEvents(List events, HMSExternalCatalog hmsExternalCatalog) { + //转换过滤 + + doExecute(events); + + hmsExternalCatalog.setLastSyncedEventId(events.get(events.size() - 1).getEventId()); + } + + @Override + protected void runAfterCatalogReady() { + List catalogIds = Env.getCurrentEnv().getCatalogMgr().getCatalogIds(); + for (Long catalogId : catalogIds) { + CatalogIf catalog = Env.getCurrentEnv().getCatalogMgr().getCatalog(catalogId); + if (catalog instanceof HMSExternalCatalog) { + HMSExternalCatalog hmsExternalCatalog = (HMSExternalCatalog) catalog; + List events = Collections.emptyList(); + try { + events = getNextHMSEvents(hmsExternalCatalog); + if (!events.isEmpty()) { + LOG.info("Events size are {} on catalog [{}]", events.size(), hmsExternalCatalog.getName()); + processEvents(events, hmsExternalCatalog); + } + } catch (MetastoreNotificationFetchException e) { + LOG.error("Failed to fetch hms events on {}. msg: ", hmsExternalCatalog.getName(), e); + } catch (Exception ex) { + LOG.error("Failed to process hive metastore [{}] events in the range of event id from {} to {}.", + hmsExternalCatalog.getName(), + events.get(0).getEventId(), events.get(events.size() - 1).getEventId(), ex); + } + } + } + } + + public static MessageDeserializer getMessageDeserializer() { + return MESSAGE_DESERIALIZER; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreNotificationException.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreNotificationException.java new file mode 100644 index 00000000000000..2bd5c4c40c904e --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreNotificationException.java @@ -0,0 +1,37 @@ +// 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.doris.datasource.hive.event; + +/** + * Utility exception class to be thrown for errors during event processing + */ +public class MetastoreNotificationException extends RuntimeException { + + public MetastoreNotificationException(String msg, Throwable cause) { + super(msg, cause); + } + + public MetastoreNotificationException(String msg) { + super(msg); + } + + public MetastoreNotificationException(Exception e) { + super(e); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreNotificationFetchException.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreNotificationFetchException.java new file mode 100644 index 00000000000000..487165eeca25cf --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreNotificationFetchException.java @@ -0,0 +1,37 @@ +// 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.doris.datasource.hive.event; + +/** + * Utility exception class to be thrown for errors during event processing + */ +public class MetastoreNotificationFetchException extends MetastoreNotificationException { + + public MetastoreNotificationFetchException(String msg, Throwable cause) { + super(msg, cause); + } + + public MetastoreNotificationFetchException(String msg) { + super(msg); + } + + public MetastoreNotificationFetchException(Exception e) { + super(e); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreTableEvent.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreTableEvent.java new file mode 100644 index 00000000000000..30d9f9b4b428a5 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreTableEvent.java @@ -0,0 +1,68 @@ +// 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.doris.datasource.hive.event; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import org.apache.hadoop.hive.metastore.api.NotificationEvent; +import org.apache.hadoop.hive.metastore.api.Table; + +import java.util.List; + +/** + * Base class for all the table events + */ +public abstract class MetastoreTableEvent extends MetastoreEvent { + // tblName from the event + protected final String tblName; + + // tbl object from the Notification event, corresponds to the before tableObj in case of alter events. + protected Table hmsTbl; + + // HivePartitionName of each event to process. for unpartition table, the partition values are empty. + // protected List hivePartitionNames = Lists.newArrayList(); + + protected MetastoreTableEvent(NotificationEvent event, String catalogName) { + super(event, catalogName); + // Preconditions.checkNotNull(dbName, "Database name cannot be null"); + tblName = Preconditions.checkNotNull(event.getTableName()); + // + // HivePartitionName hivePartitionName = new HivePartitionName(dbName, tblName, Lists.newArrayList()); + // hivePartitionNames.add(hivePartitionName); + } + + /** + * Returns a list of parameters that are set by Hive for tables/partitions that can be + * ignored to determine if the alter table/partition event is a trivial one. + */ + private static final List PARAMETERS_TO_IGNORE = + new ImmutableList.Builder() + .add("transient_lastDdlTime") + .add("numFilesErasureCoded") + .add("numFiles") + .add("comment") + .build(); + + /** + * According to the current processing method, each event only needs to process one {@link HivePartitionName}. + */ + // protected HivePartitionName getHivePartitionName() { + // return hivePartitionNames.get(0); + // } +} From 7cbedbeaa0fddf700c28aa2a292b6df2531d7952 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Tue, 27 Dec 2022 17:08:37 +0800 Subject: [PATCH 02/25] hive event init --- .../doris/datasource/HMSExternalCatalog.java | 108 +++++++++--------- 1 file changed, 54 insertions(+), 54 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java index ab642ae2c38d6f..d4ff9a8dbac1f6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java @@ -77,32 +77,39 @@ public String getHiveMetastoreUris() { } @Override - public void notifyPropertiesUpdated() { - initLocalObjectsImpl(); - } - - @Override - public List listDatabaseNames(SessionContext ctx) { - makeSureInitialized(); - return Lists.newArrayList(dbNameToId.keySet()); - } - - @Override - public List listTableNames(SessionContext ctx, String dbName) { - makeSureInitialized(); - HMSExternalDatabase hmsExternalDatabase = (HMSExternalDatabase) idToDb.get(dbNameToId.get(dbName)); - if (hmsExternalDatabase != null && hmsExternalDatabase.isInitialized()) { - List names = Lists.newArrayList(); - hmsExternalDatabase.getTables().forEach(table -> names.add(table.getName())); - return names; - } else { - return client.getAllTables(getRealTableName(dbName)); + protected void init() { + Map tmpDbNameToId = Maps.newConcurrentMap(); + Map tmpIdToDb = Maps.newConcurrentMap(); + InitCatalogLog initCatalogLog = new InitCatalogLog(); + initCatalogLog.setCatalogId(id); + initCatalogLog.setType(InitCatalogLog.Type.HMS); + List allDatabases = client.getAllDatabases(); + // Update the db name to id map. + for (String dbName : allDatabases) { + long dbId; + if (dbNameToId != null && dbNameToId.containsKey(dbName)) { + dbId = dbNameToId.get(dbName); + tmpDbNameToId.put(dbName, dbId); + ExternalDatabase db = idToDb.get(dbId); + db.setUnInitialized(invalidCacheInInit); + tmpIdToDb.put(dbId, db); + initCatalogLog.addRefreshDb(dbId); + } else { + dbId = Env.getCurrentEnv().getNextId(); + tmpDbNameToId.put(dbName, dbId); + HMSExternalDatabase db = new HMSExternalDatabase(this, dbId, dbName); + tmpIdToDb.put(dbId, db); + initCatalogLog.addCreateDb(dbId, dbName); + } } + dbNameToId = tmpDbNameToId; + idToDb = tmpIdToDb; + Env.getCurrentEnv().getEditLog().logInitCatalog(initCatalogLog); } @Override - public boolean tableExist(SessionContext ctx, String dbName, String tblName) { - return client.tableExists(getRealTableName(dbName), tblName); + public void notifyPropertiesUpdated() { + initLocalObjectsImpl(); } @Override @@ -136,34 +143,32 @@ protected void initLocalObjectsImpl() { } @Override - protected void init() { - Map tmpDbNameToId = Maps.newConcurrentMap(); - Map tmpIdToDb = Maps.newConcurrentMap(); - InitCatalogLog initCatalogLog = new InitCatalogLog(); - initCatalogLog.setCatalogId(id); - initCatalogLog.setType(InitCatalogLog.Type.HMS); - List allDatabases = client.getAllDatabases(); - // Update the db name to id map. - for (String dbName : allDatabases) { - long dbId; - if (dbNameToId != null && dbNameToId.containsKey(dbName)) { - dbId = dbNameToId.get(dbName); - tmpDbNameToId.put(dbName, dbId); - ExternalDatabase db = idToDb.get(dbId); - db.setUnInitialized(invalidCacheInInit); - tmpIdToDb.put(dbId, db); - initCatalogLog.addRefreshDb(dbId); - } else { - dbId = Env.getCurrentEnv().getNextId(); - tmpDbNameToId.put(dbName, dbId); - HMSExternalDatabase db = new HMSExternalDatabase(this, dbId, dbName); - tmpIdToDb.put(dbId, db); - initCatalogLog.addCreateDb(dbId, dbName); - } + public List listDatabaseNames(SessionContext ctx) { + makeSureInitialized(); + return Lists.newArrayList(dbNameToId.keySet()); + } + + @Override + public List listTableNames(SessionContext ctx, String dbName) { + makeSureInitialized(); + HMSExternalDatabase hmsExternalDatabase = (HMSExternalDatabase) idToDb.get(dbNameToId.get(dbName)); + if (hmsExternalDatabase != null && hmsExternalDatabase.isInitialized()) { + List names = Lists.newArrayList(); + hmsExternalDatabase.getTables().forEach(table -> names.add(table.getName())); + return names; + } else { + return client.getAllTables(getRealTableName(dbName)); } - dbNameToId = tmpDbNameToId; - idToDb = tmpIdToDb; - Env.getCurrentEnv().getEditLog().logInitCatalog(initCatalogLog); + } + + @Override + public boolean tableExist(SessionContext ctx, String dbName, String tblName) { + return client.tableExists(getRealTableName(dbName), tblName); + } + + public PooledHiveMetaStoreClient getClient() { + makeSureInitialized(); + return client; } @Override @@ -179,11 +184,6 @@ public List getSchema(String dbName, String tblName) { return tmpSchema; } - public PooledHiveMetaStoreClient getClient() { - makeSureInitialized(); - return client; - } - public void setLastSyncedEventId(long lastSyncedEventId) { this.lastSyncedEventId = lastSyncedEventId; } From c787a4644871e27cfa96b5db5fe47ac101968184 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Tue, 27 Dec 2022 17:18:17 +0800 Subject: [PATCH 03/25] hive event init --- fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java index a8a1c8b78fa495..5fa85986192a95 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java @@ -126,6 +126,7 @@ import org.apache.doris.datasource.EsExternalCatalog; import org.apache.doris.datasource.ExternalMetaCacheMgr; import org.apache.doris.datasource.InternalCatalog; +import org.apache.doris.datasource.hive.event.MetastoreEventsProcessor; import org.apache.doris.deploy.DeployManager; import org.apache.doris.deploy.impl.AmbariDeployManager; import org.apache.doris.deploy.impl.K8sDeployManager; @@ -316,6 +317,7 @@ public class Env { private DeleteHandler deleteHandler; private DbUsedDataQuotaInfoCollector dbUsedDataQuotaInfoCollector; private PartitionInMemoryInfoCollector partitionInMemoryInfoCollector; + private MetastoreEventsProcessor metastoreEventsProcessor; private MasterDaemon labelCleaner; // To clean old LabelInfo, ExportJobInfos private MasterDaemon txnCleaner; // To clean aborted or timeout txns @@ -549,6 +551,7 @@ private Env(boolean isCheckpointCatalog) { this.deleteHandler = new DeleteHandler(); this.dbUsedDataQuotaInfoCollector = new DbUsedDataQuotaInfoCollector(); this.partitionInMemoryInfoCollector = new PartitionInMemoryInfoCollector(); + this.metastoreEventsProcessor = new MetastoreEventsProcessor(); this.replayedJournalId = new AtomicLong(0L); this.isElectable = false; @@ -1402,6 +1405,8 @@ private void startMasterOnlyDaemonThreads() { if (Config.enable_fqdn_mode) { fqdnManager.start(); } + // TODO: 2022/12/27 判断是否开启 + metastoreEventsProcessor.start(); } // start threads that should running on all FE From 82ed858a651c3cd284e068437eca9b08f7a8031b Mon Sep 17 00:00:00 2001 From: zhangdong Date: Wed, 28 Dec 2022 12:07:14 +0800 Subject: [PATCH 04/25] hive event init --- .../org/apache/doris/datasource/HMSExternalCatalog.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java index d4ff9a8dbac1f6..83e69fcf654958 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java @@ -33,6 +33,7 @@ import com.google.common.collect.Maps; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.CurrentNotificationEventId; import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.NotificationEventResponse; import org.apache.hadoop.security.UserGroupInformation; @@ -205,6 +206,11 @@ public NotificationEventResponse getNextEventResponse(String catalogName) } public long getCurrentEventId() { - return client.getCurrentNotificationEventId().getEventId(); + CurrentNotificationEventId currentNotificationEventId = client.getCurrentNotificationEventId(); + if (currentNotificationEventId == null) { + LOG.error("获取currentNotificationEventId为null"); + return -1; + } + return currentNotificationEventId.getEventId(); } } From 0998f26cf23a4d8dc8224c82c02c95d4637ed18b Mon Sep 17 00:00:00 2001 From: zhangdong Date: Wed, 28 Dec 2022 13:42:17 +0800 Subject: [PATCH 05/25] hive event init --- .../java/org/apache/doris/datasource/HMSExternalCatalog.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java index 83e69fcf654958..8d636b2637befe 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java @@ -69,7 +69,7 @@ public HMSExternalCatalog( catalogProperty = new CatalogProperty(resource, props); // TODO: 2022/12/27 判断是否开启同步 // if (enableHmsEventsIncrementalSync) { - setLastSyncedEventId(getCurrentEventId()); + // setLastSyncedEventId(getCurrentEventId()); // } } From f3f18bcbaa1b94952442b7010fb0a43147c1bc68 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Wed, 28 Dec 2022 14:42:39 +0800 Subject: [PATCH 06/25] hive event init --- .../datasource/hive/event/MetastoreEventsProcessor.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java index ac51edc4796c8e..b28884f11a48ee 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java @@ -137,9 +137,11 @@ protected void runAfterCatalogReady() { } catch (MetastoreNotificationFetchException e) { LOG.error("Failed to fetch hms events on {}. msg: ", hmsExternalCatalog.getName(), e); } catch (Exception ex) { - LOG.error("Failed to process hive metastore [{}] events in the range of event id from {} to {}.", - hmsExternalCatalog.getName(), - events.get(0).getEventId(), events.get(events.size() - 1).getEventId(), ex); + // LOG.error("Failed to process hive metastore [{}] events in the range of event id from {} to {}.", + // hmsExternalCatalog.getName(), + // events.get(0).getEventId(), events.get(events.size() - 1).getEventId(), ex); + LOG.error("Failed to process hive metastore [{}] events .", + hmsExternalCatalog.getName(), ex); } } } From e20eafce70d40f76ca24a1ce2d716f0d648e4827 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Wed, 28 Dec 2022 14:47:45 +0800 Subject: [PATCH 07/25] hive event init --- .../org/apache/doris/datasource/HMSExternalCatalog.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java index 8d636b2637befe..81ff19af1891fd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java @@ -192,8 +192,9 @@ public void setLastSyncedEventId(long lastSyncedEventId) { public NotificationEventResponse getNextEventResponse(String catalogName) throws MetastoreNotificationFetchException { if (lastSyncedEventId == -1) { - lastSyncedEventId = getCurrentEventId(); LOG.error("Last synced event id is null when pulling events on catalog [{}]", catalogName); + lastSyncedEventId = getCurrentEventId(); + // LOG.error("Last synced event id is null when pulling events on catalog [{}]", catalogName); return null; } @@ -206,6 +207,10 @@ public NotificationEventResponse getNextEventResponse(String catalogName) } public long getCurrentEventId() { + if (client == null) { + LOG.error("client为空===================="); + return -1; + } CurrentNotificationEventId currentNotificationEventId = client.getCurrentNotificationEventId(); if (currentNotificationEventId == null) { LOG.error("获取currentNotificationEventId为null"); From 944e9eeea2541bb5e7c45d2a9b4f01100da645f9 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Wed, 28 Dec 2022 14:51:59 +0800 Subject: [PATCH 08/25] hive event init --- .../doris/datasource/hive/event/MetastoreEventsProcessor.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java index b28884f11a48ee..606c72c341cc2c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java @@ -137,9 +137,6 @@ protected void runAfterCatalogReady() { } catch (MetastoreNotificationFetchException e) { LOG.error("Failed to fetch hms events on {}. msg: ", hmsExternalCatalog.getName(), e); } catch (Exception ex) { - // LOG.error("Failed to process hive metastore [{}] events in the range of event id from {} to {}.", - // hmsExternalCatalog.getName(), - // events.get(0).getEventId(), events.get(events.size() - 1).getEventId(), ex); LOG.error("Failed to process hive metastore [{}] events .", hmsExternalCatalog.getName(), ex); } From bc1446d7486ae197ffacbde7190d44a482cbe6d6 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Wed, 28 Dec 2022 15:24:45 +0800 Subject: [PATCH 09/25] hive event init --- .../java/org/apache/doris/datasource/HMSExternalCatalog.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java index 81ff19af1891fd..dcdbe156822534 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java @@ -191,6 +191,7 @@ public void setLastSyncedEventId(long lastSyncedEventId) { public NotificationEventResponse getNextEventResponse(String catalogName) throws MetastoreNotificationFetchException { + makeSureInitialized(); if (lastSyncedEventId == -1) { LOG.error("Last synced event id is null when pulling events on catalog [{}]", catalogName); lastSyncedEventId = getCurrentEventId(); @@ -207,6 +208,7 @@ public NotificationEventResponse getNextEventResponse(String catalogName) } public long getCurrentEventId() { + makeSureInitialized(); if (client == null) { LOG.error("client为空===================="); return -1; From 3b6da9b1279b5e074ab5d0cc088966b7579cfea4 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Wed, 28 Dec 2022 15:50:57 +0800 Subject: [PATCH 10/25] hive event init --- .../org/apache/doris/datasource/HMSExternalCatalog.java | 3 ++- .../datasource/hive/event/MetastoreEventsProcessor.java | 9 +++++---- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java index dcdbe156822534..1f1ef018516aee 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java @@ -200,8 +200,9 @@ public NotificationEventResponse getNextEventResponse(String catalogName) } long currentEventId = getCurrentEventId(); + LOG.error("本次getNextEventResponse的currentEventId为{},lastSyncedEventId为{}", currentEventId, lastSyncedEventId); if (currentEventId == lastSyncedEventId) { - LOG.info("Event id not updated when pulling events on catalog [{}]", catalogName); + LOG.error("Event id not updated when pulling events on catalog [{}]", catalogName); return null; } return client.getNextNotification(lastSyncedEventId, Config.hms_events_batch_size_per_rpc, null); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java index 606c72c341cc2c..a1bcf64c9d84b5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java @@ -77,7 +77,7 @@ public MetastoreEventsProcessor() { * {@link Config#hms_events_batch_size_per_rpc} */ private List getNextHMSEvents(HMSExternalCatalog hmsExternalCatalog) { - LOG.info("Start to pull events on catalog [{}]", hmsExternalCatalog.getName()); + LOG.error("Start to pull events on catalog [{}]", hmsExternalCatalog.getName()); NotificationEventResponse response = hmsExternalCatalog.getNextEventResponse(hmsExternalCatalog.getName()); if (response == null) { @@ -104,8 +104,9 @@ private List getNextHMSEvents(HMSExternalCatalog hmsExternalC private void doExecute(List events) { for (NotificationEvent event : events) { - LOG.info("event消息内容:{}", event.toString()); - // LOG.info("收到hive event消息,id:为{},时间为:{},db为:{},table为:{},type为:{}",event.getEventId(),); + // LOG.info("event消息内容:{}", event.toString()); + LOG.error("收到hive event消息,id:为{},时间为:{},db为:{},table为:{},type为:{}", event.getEventId(), event.getEventTime(), + event.getDbName(), event.getTableName(), event.getEventType()); } } @@ -131,7 +132,7 @@ protected void runAfterCatalogReady() { try { events = getNextHMSEvents(hmsExternalCatalog); if (!events.isEmpty()) { - LOG.info("Events size are {} on catalog [{}]", events.size(), hmsExternalCatalog.getName()); + LOG.error("Events size are {} on catalog [{}]", events.size(), hmsExternalCatalog.getName()); processEvents(events, hmsExternalCatalog); } } catch (MetastoreNotificationFetchException e) { From 63691c7d7e9c29cd994eda0d15104ae1b14c46c4 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Wed, 28 Dec 2022 15:55:15 +0800 Subject: [PATCH 11/25] hive event init --- .../doris/datasource/hive/event/MetastoreEventsProcessor.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java index a1bcf64c9d84b5..cc40ffb76b26dd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java @@ -105,8 +105,8 @@ private List getNextHMSEvents(HMSExternalCatalog hmsExternalC private void doExecute(List events) { for (NotificationEvent event : events) { // LOG.info("event消息内容:{}", event.toString()); - LOG.error("收到hive event消息,id:为{},时间为:{},db为:{},table为:{},type为:{}", event.getEventId(), event.getEventTime(), - event.getDbName(), event.getTableName(), event.getEventType()); + LOG.error("收到hive event消息,id:为{},时间为:{},db为:{},table为:{},type为:{}", event.getEventId(), + event.getEventTime(), event.getDbName(), event.getTableName(), event.getEventType()); } } From b71956a6b18401826d082da4f8edab1b46b5df3c Mon Sep 17 00:00:00 2001 From: zhangdong Date: Wed, 28 Dec 2022 17:33:32 +0800 Subject: [PATCH 12/25] hive event init --- .../java/org/apache/doris/datasource/HMSExternalCatalog.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java index 1f1ef018516aee..8b4807a742a5af 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java @@ -53,7 +53,7 @@ public class HMSExternalCatalog extends ExternalCatalog { private static final int MAX_CLIENT_POOL_SIZE = 8; protected PooledHiveMetaStoreClient client; // Record the latest synced event id when processing hive events - private long lastSyncedEventId = -1; + private long lastSyncedEventId = 2L; /** * Default constructor for HMSExternalCatalog. @@ -192,7 +192,7 @@ public void setLastSyncedEventId(long lastSyncedEventId) { public NotificationEventResponse getNextEventResponse(String catalogName) throws MetastoreNotificationFetchException { makeSureInitialized(); - if (lastSyncedEventId == -1) { + if (lastSyncedEventId == 2L) { LOG.error("Last synced event id is null when pulling events on catalog [{}]", catalogName); lastSyncedEventId = getCurrentEventId(); // LOG.error("Last synced event id is null when pulling events on catalog [{}]", catalogName); From bafc2a740936861c6fea4d5036d41d530a818f66 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Wed, 28 Dec 2022 17:47:58 +0800 Subject: [PATCH 13/25] hive event init --- fe/fe-core/src/main/java/org/apache/doris/common/Config.java | 2 +- .../java/org/apache/doris/datasource/HMSExternalCatalog.java | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/Config.java b/fe/fe-core/src/main/java/org/apache/doris/common/Config.java index e3edff3f5d1fbd..34ae0d53d338c4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/Config.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/Config.java @@ -1946,6 +1946,6 @@ public class Config extends ConfigBase { * HMS polling interval in milliseconds. */ @ConfField - public static int hms_events_polling_interval_ms = 5000; + public static int hms_events_polling_interval_ms = 20000; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java index 8b4807a742a5af..e865b44b61361f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java @@ -53,7 +53,7 @@ public class HMSExternalCatalog extends ExternalCatalog { private static final int MAX_CLIENT_POOL_SIZE = 8; protected PooledHiveMetaStoreClient client; // Record the latest synced event id when processing hive events - private long lastSyncedEventId = 2L; + private long lastSyncedEventId; /** * Default constructor for HMSExternalCatalog. @@ -63,6 +63,7 @@ public HMSExternalCatalog( this.id = catalogId; this.name = name; this.type = "hms"; + this.lastSyncedEventId = -1l; if (resource == null) { props.putAll(HMSResource.getPropertiesFromDLF()); } @@ -192,7 +193,7 @@ public void setLastSyncedEventId(long lastSyncedEventId) { public NotificationEventResponse getNextEventResponse(String catalogName) throws MetastoreNotificationFetchException { makeSureInitialized(); - if (lastSyncedEventId == 2L) { + if (lastSyncedEventId == -1l) { LOG.error("Last synced event id is null when pulling events on catalog [{}]", catalogName); lastSyncedEventId = getCurrentEventId(); // LOG.error("Last synced event id is null when pulling events on catalog [{}]", catalogName); From 0b5987cb6dfcbcd0458540e1c2c37439c5eae64e Mon Sep 17 00:00:00 2001 From: zhangdong Date: Wed, 28 Dec 2022 17:55:25 +0800 Subject: [PATCH 14/25] hive event init --- .../java/org/apache/doris/datasource/HMSExternalCatalog.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java index e865b44b61361f..545c17d60578eb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java @@ -63,7 +63,7 @@ public HMSExternalCatalog( this.id = catalogId; this.name = name; this.type = "hms"; - this.lastSyncedEventId = -1l; + this.lastSyncedEventId = -1L; if (resource == null) { props.putAll(HMSResource.getPropertiesFromDLF()); } @@ -193,7 +193,7 @@ public void setLastSyncedEventId(long lastSyncedEventId) { public NotificationEventResponse getNextEventResponse(String catalogName) throws MetastoreNotificationFetchException { makeSureInitialized(); - if (lastSyncedEventId == -1l) { + if (lastSyncedEventId == -1L) { LOG.error("Last synced event id is null when pulling events on catalog [{}]", catalogName); lastSyncedEventId = getCurrentEventId(); // LOG.error("Last synced event id is null when pulling events on catalog [{}]", catalogName); From ed69eb4bc6d9981cbee4ecf0f9c33daf9bec0ecb Mon Sep 17 00:00:00 2001 From: zhangdong Date: Wed, 28 Dec 2022 18:14:04 +0800 Subject: [PATCH 15/25] hive event init --- .../apache/doris/datasource/HMSExternalCatalog.java | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java index 545c17d60578eb..8c44253b918b3c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java @@ -53,7 +53,7 @@ public class HMSExternalCatalog extends ExternalCatalog { private static final int MAX_CLIENT_POOL_SIZE = 8; protected PooledHiveMetaStoreClient client; // Record the latest synced event id when processing hive events - private long lastSyncedEventId; + private long lastSyncedEventId = -1; /** * Default constructor for HMSExternalCatalog. @@ -63,15 +63,12 @@ public HMSExternalCatalog( this.id = catalogId; this.name = name; this.type = "hms"; - this.lastSyncedEventId = -1L; if (resource == null) { props.putAll(HMSResource.getPropertiesFromDLF()); } catalogProperty = new CatalogProperty(resource, props); - // TODO: 2022/12/27 判断是否开启同步 - // if (enableHmsEventsIncrementalSync) { - // setLastSyncedEventId(getCurrentEventId()); - // } + setLastSyncedEventId(getCurrentEventId()); + LOG.error("new HMSExternalCatalog(),lastSyncedEventId = {}", lastSyncedEventId); } public String getHiveMetastoreUris() { @@ -193,7 +190,7 @@ public void setLastSyncedEventId(long lastSyncedEventId) { public NotificationEventResponse getNextEventResponse(String catalogName) throws MetastoreNotificationFetchException { makeSureInitialized(); - if (lastSyncedEventId == -1L) { + if (lastSyncedEventId == -1) { LOG.error("Last synced event id is null when pulling events on catalog [{}]", catalogName); lastSyncedEventId = getCurrentEventId(); // LOG.error("Last synced event id is null when pulling events on catalog [{}]", catalogName); From d12aa13f76a08b31c0dc4a1677e5414ba4d0e211 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Wed, 28 Dec 2022 18:58:20 +0800 Subject: [PATCH 16/25] hive event init --- .../org/apache/doris/datasource/HMSExternalCatalog.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java index 8c44253b918b3c..82924473dfedd0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java @@ -53,7 +53,7 @@ public class HMSExternalCatalog extends ExternalCatalog { private static final int MAX_CLIENT_POOL_SIZE = 8; protected PooledHiveMetaStoreClient client; // Record the latest synced event id when processing hive events - private long lastSyncedEventId = -1; + private long lastSyncedEventId; /** * Default constructor for HMSExternalCatalog. @@ -67,8 +67,8 @@ public HMSExternalCatalog( props.putAll(HMSResource.getPropertiesFromDLF()); } catalogProperty = new CatalogProperty(resource, props); - setLastSyncedEventId(getCurrentEventId()); - LOG.error("new HMSExternalCatalog(),lastSyncedEventId = {}", lastSyncedEventId); + // setLastSyncedEventId(getCurrentEventId()); + // LOG.error("new HMSExternalCatalog(),lastSyncedEventId = {}", lastSyncedEventId); } public String getHiveMetastoreUris() { @@ -190,7 +190,7 @@ public void setLastSyncedEventId(long lastSyncedEventId) { public NotificationEventResponse getNextEventResponse(String catalogName) throws MetastoreNotificationFetchException { makeSureInitialized(); - if (lastSyncedEventId == -1) { + if (lastSyncedEventId == 0) { LOG.error("Last synced event id is null when pulling events on catalog [{}]", catalogName); lastSyncedEventId = getCurrentEventId(); // LOG.error("Last synced event id is null when pulling events on catalog [{}]", catalogName); From 4fa03aafa57c4ae26544ea72636378ec0edd5479 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Fri, 30 Dec 2022 10:23:55 +0800 Subject: [PATCH 17/25] hive event init --- .../org/apache/doris/catalog/DatabaseIf.java | 2 + .../apache/doris/catalog/RefreshManager.java | 21 +++ .../catalog/external/ExternalDatabase.java | 5 + .../catalog/external/HMSExternalDatabase.java | 9 + .../apache/doris/datasource/CatalogMgr.java | 51 +++++- .../doris/datasource/HMSExternalCatalog.java | 20 ++- .../hive/event/AddPartitionEvent.java | 86 ---------- .../hive/event/AlterPartitionEvent.java | 97 ----------- .../hive/event/AlterTableEvent.java | 125 -------------- .../datasource/hive/event/BatchEvent.java | 158 ------------------ .../hive/event/CreateTableEvent.java | 60 ------- .../hive/event/DropPartitionEvent.java | 103 ------------ .../datasource/hive/event/DropTableEvent.java | 12 +- .../datasource/hive/event/EventFactory.java | 5 - .../datasource/hive/event/InsertEvent.java | 104 ------------ .../datasource/hive/event/MetastoreEvent.java | 17 +- .../hive/event/MetastoreEventFactory.java | 40 +---- .../hive/event/MetastoreEventsProcessor.java | 28 +++- .../hive/event/MetastoreTableEvent.java | 22 +-- .../org/apache/doris/persist/EditLog.java | 9 + .../apache/doris/persist/OperationType.java | 2 + 21 files changed, 146 insertions(+), 830 deletions(-) delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/AddPartitionEvent.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/AlterPartitionEvent.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/AlterTableEvent.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/BatchEvent.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/CreateTableEvent.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/DropPartitionEvent.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/InsertEvent.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/DatabaseIf.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/DatabaseIf.java index ffd271d84ea50d..f0dfa53484f4de 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/DatabaseIf.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/DatabaseIf.java @@ -214,4 +214,6 @@ default OlapTable getOlapTableOrAnalysisException(String tableName) throws Analy } return (OlapTable) table; } + + void dropTable(String tableName); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/RefreshManager.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/RefreshManager.java index b8d69299940053..b224e8df678d78 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/RefreshManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/RefreshManager.java @@ -167,4 +167,25 @@ private void refreshExternalCtlTable(String dbName, String tableName, CatalogIf log.setTableId(table.getId()); Env.getCurrentEnv().getEditLog().logRefreshExternalTable(log); } + + private void dropExternalCtlTable(String dbName, String tableName, CatalogIf catalog) throws DdlException { + if (!(catalog instanceof ExternalCatalog)) { + throw new DdlException("Only support drop ExternalCatalog Tables"); + } + DatabaseIf db = catalog.getDbNullable(dbName); + if (db == null) { + throw new DdlException("Database " + dbName + " does not exist in catalog " + catalog.getName()); + } + + TableIf table = db.getTableNullable(tableName); + if (table == null) { + throw new DdlException("Table " + tableName + " does not exist in db " + dbName); + } + Env.getCurrentEnv().getExtMetaCacheMgr().invalidateTableCache(catalog.getId(), dbName, tableName); + ExternalObjectLog log = new ExternalObjectLog(); + log.setCatalogId(catalog.getId()); + log.setDbId(db.getId()); + log.setTableId(table.getId()); + Env.getCurrentEnv().getEditLog().logRefreshExternalTable(log); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/ExternalDatabase.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/ExternalDatabase.java index 6ae8594c07765c..65c027713e2b02 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/ExternalDatabase.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/ExternalDatabase.java @@ -258,4 +258,9 @@ public static ExternalDatabase read(DataInput in) throws IOException { @Override public void gsonPostProcess() throws IOException {} + + @Override + public void dropTable(String tableName) { + throw new NotImplementedException(); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalDatabase.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalDatabase.java index decef86caa1493..c23fbe637ba933 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalDatabase.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalDatabase.java @@ -170,4 +170,13 @@ public void addTableForTest(HMSExternalTable tbl) { idToTbl.put(tbl.getId(), tbl); tableNameToId.put(tbl.getName(), tbl.getId()); } + + @Override + public void dropTable(String tableName) { + Long tableId = tableNameToId.remove(tableName); + if (tableId == null) { + LOG.warn("drop table [{}] failed", tableName); + } + idToTbl.remove(tableId); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java index 255b63526f7808..82ca3f0ab75179 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java @@ -28,6 +28,7 @@ import org.apache.doris.catalog.Env; import org.apache.doris.catalog.Resource; import org.apache.doris.catalog.Resource.ReferenceType; +import org.apache.doris.catalog.TableIf; import org.apache.doris.catalog.external.ExternalDatabase; import org.apache.doris.catalog.external.ExternalTable; import org.apache.doris.cluster.ClusterNamespace; @@ -435,13 +436,17 @@ public ShowResultSet showCreateCatalog(ShowCreateCatalogStmt showStmt) throws An * Refresh the catalog meta and write the meta log. */ public void refreshCatalog(RefreshCatalogStmt stmt) throws UserException { + CatalogIf catalog = nameToCatalog.get(stmt.getCatalogName()); + if (catalog == null) { + throw new DdlException("No catalog found with name: " + stmt.getCatalogName()); + } + CatalogLog log = CatalogFactory.constructorCatalogLog(catalog.getId(), stmt); + refreshCatalog(log); + } + + public void refreshCatalog(CatalogLog log) { writeLock(); try { - CatalogIf catalog = nameToCatalog.get(stmt.getCatalogName()); - if (catalog == null) { - throw new DdlException("No catalog found with name: " + stmt.getCatalogName()); - } - CatalogLog log = CatalogFactory.constructorCatalogLog(catalog.getId(), stmt); replayRefreshCatalog(log); Env.getCurrentEnv().getEditLog().logCatalogLog(OperationType.OP_REFRESH_CATALOG, log); } finally { @@ -477,7 +482,7 @@ public void replayDropCatalog(CatalogLog log) { /** * Reply for refresh catalog event. */ - public void replayRefreshCatalog(CatalogLog log) throws DdlException { + public void replayRefreshCatalog(CatalogLog log) { writeLock(); try { unprotectedRefreshCatalog(log.getCatalogId(), log.isInvalidCache()); @@ -550,6 +555,40 @@ public void replayRefreshExternalTable(ExternalObjectLog log) { .invalidateTableCache(catalog.getId(), db.getFullName(), table.getName()); } + public void dropExternalTable(String dbName, String tableName, String catalogName) throws DdlException { + CatalogIf catalog = nameToCatalog.get(catalogName); + if (catalog == null) { + throw new DdlException("No catalog found with name: " + catalogName); + } + if (!(catalog instanceof ExternalCatalog)) { + throw new DdlException("Only support drop ExternalCatalog Tables"); + } + DatabaseIf db = catalog.getDbNullable(dbName); + if (db == null) { + throw new DdlException("Database " + dbName + " does not exist in catalog " + catalog.getName()); + } + + TableIf table = db.getTableNullable(tableName); + if (table == null) { + throw new DdlException("Table " + tableName + " does not exist in db " + dbName); + } + ExternalObjectLog log = new ExternalObjectLog(); + log.setCatalogId(catalog.getId()); + log.setDbId(db.getId()); + log.setTableId(table.getId()); + replayDropExternalTable(log); + Env.getCurrentEnv().getEditLog().logDropExternalTable(log); + } + + // TODO: 2022/12/29 check null ? + public void replayDropExternalTable(ExternalObjectLog log) { + ExternalCatalog catalog = (ExternalCatalog) idToCatalog.get(log.getCatalogId()); + ExternalDatabase db = catalog.getDbForReplay(log.getDbId()); + ExternalTable table = db.getTableForReplay(log.getTableId()); + Env.getCurrentEnv().getExtMetaCacheMgr() + .invalidateTableCache(catalog.getId(), db.getFullName(), table.getName()); + } + @Override public void write(DataOutput out) throws IOException { String json = GsonUtils.GSON.toJson(this); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java index 82924473dfedd0..b024550bb80dd4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java @@ -187,25 +187,35 @@ public void setLastSyncedEventId(long lastSyncedEventId) { this.lastSyncedEventId = lastSyncedEventId; } - public NotificationEventResponse getNextEventResponse(String catalogName) + public NotificationEventResponse getNextEventResponse(HMSExternalCatalog hmsExternalCatalog) throws MetastoreNotificationFetchException { makeSureInitialized(); - if (lastSyncedEventId == 0) { - LOG.error("Last synced event id is null when pulling events on catalog [{}]", catalogName); + if (lastSyncedEventId <= 0) { lastSyncedEventId = getCurrentEventId(); - // LOG.error("Last synced event id is null when pulling events on catalog [{}]", catalogName); + refreshCatalog(hmsExternalCatalog); + LOG.error( + "First pulling events on catalog [{}],refreshCatalog and init lastSyncedEventId," + + "lastSyncedEventId is [{}]", + hmsExternalCatalog.getName(), lastSyncedEventId); return null; } long currentEventId = getCurrentEventId(); LOG.error("本次getNextEventResponse的currentEventId为{},lastSyncedEventId为{}", currentEventId, lastSyncedEventId); if (currentEventId == lastSyncedEventId) { - LOG.error("Event id not updated when pulling events on catalog [{}]", catalogName); + LOG.error("Event id not updated when pulling events on catalog [{}]", hmsExternalCatalog.getName()); return null; } return client.getNextNotification(lastSyncedEventId, Config.hms_events_batch_size_per_rpc, null); } + private void refreshCatalog(HMSExternalCatalog hmsExternalCatalog) { + CatalogLog log = new CatalogLog(); + log.setCatalogId(hmsExternalCatalog.getId()); + log.setInvalidCache(true); + Env.getCurrentEnv().getCatalogMgr().refreshCatalog(log); + } + public long getCurrentEventId() { makeSureInitialized(); if (client == null) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/AddPartitionEvent.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/AddPartitionEvent.java deleted file mode 100644 index 2f69cce2558889..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/AddPartitionEvent.java +++ /dev/null @@ -1,86 +0,0 @@ -// 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.doris.datasource.hive.event; - -import com.google.common.base.Preconditions; -import com.google.common.collect.Lists; -import org.apache.hadoop.hive.metastore.api.NotificationEvent; -import org.apache.hadoop.hive.metastore.api.Partition; -import org.apache.hadoop.hive.metastore.messaging.AddPartitionMessage; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; - -import java.util.List; - -/** - * MetastoreEvent for ADD_PARTITION event type - */ -public class AddPartitionEvent extends MetastoreTableEvent { - private static final Logger LOG = LogManager.getLogger(AddPartitionEvent.class); - - private final Partition addedPartition; - - /** - * Prevent instantiation from outside should use MetastoreEventFactory instead - */ - private AddPartitionEvent(NotificationEvent event, - Partition addedPartition, - String catalogName) { - super(event, catalogName); - Preconditions.checkState(getEventType().equals(MetastoreEventType.ADD_PARTITION)); - if (event.getMessage() == null) { - throw new IllegalStateException(debugString("Event message is null")); - } - - try { - AddPartitionMessage addPartitionMessage = - MetastoreEventsProcessor.getMessageDeserializer() - .getAddPartitionMessage(event.getMessage()); - this.addedPartition = addedPartition; - hmsTbl = addPartitionMessage.getTableObj(); - // hivePartitionNames.clear(); - // List partitionColNames = hmsTbl.getPartitionKeys().stream() - // .map(FieldSchema::getName).collect(Collectors.toList()); - // hivePartitionNames.add(HivePartitionName.of(dbName, tblName, - // FileUtils.makePartName(partitionColNames, addedPartition.getValues()))); - } catch (Exception ex) { - throw new MetastoreNotificationException(ex); - } - } - - protected static List getEvents(NotificationEvent event, - String catalogName) { - List addPartitionEvents = Lists.newArrayList(); - try { - AddPartitionMessage addPartitionMessage = - MetastoreEventsProcessor.getMessageDeserializer() - .getAddPartitionMessage(event.getMessage()); - addPartitionMessage.getPartitionObjs().forEach(partition -> - addPartitionEvents.add(new AddPartitionEvent(event, partition, catalogName))); - } catch (Exception ex) { - throw new MetastoreNotificationException(ex); - } - return addPartitionEvents; - } - - @Override - protected void process() throws MetastoreNotificationException { - throw new UnsupportedOperationException("Unsupported event type: " + getEventType()); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/AlterPartitionEvent.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/AlterPartitionEvent.java deleted file mode 100644 index b038b7a834fdd2..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/AlterPartitionEvent.java +++ /dev/null @@ -1,97 +0,0 @@ -// 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.doris.datasource.hive.event; - -import com.google.common.base.Preconditions; -import com.google.common.collect.Lists; -import org.apache.hadoop.hive.metastore.api.NotificationEvent; -import org.apache.hadoop.hive.metastore.api.Partition; -import org.apache.hadoop.hive.metastore.messaging.AlterPartitionMessage; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; - -import java.util.List; - -/** - * MetastoreEvent for ALTER_PARTITION event type - */ -public class AlterPartitionEvent extends MetastoreTableEvent { - private static final Logger LOG = LogManager.getLogger(AlterPartitionEvent.class); - - // the Partition object before alter operation, as parsed from the NotificationEvent - private final Partition partitionBefore; - // the Partition object after alter operation, as parsed from the NotificationEvent - private final Partition partitionAfter; - - private AlterPartitionEvent(NotificationEvent event, String catalogName) { - super(event, catalogName); - Preconditions.checkState(getEventType() == MetastoreEventType.ALTER_PARTITION); - Preconditions.checkNotNull(event.getMessage()); - AlterPartitionMessage alterPartitionMessage = - MetastoreEventsProcessor.getMessageDeserializer() - .getAlterPartitionMessage(event.getMessage()); - - try { - partitionBefore = Preconditions.checkNotNull(alterPartitionMessage.getPtnObjBefore()); - partitionAfter = Preconditions.checkNotNull(alterPartitionMessage.getPtnObjAfter()); - hmsTbl = alterPartitionMessage.getTableObj(); - } catch (Exception e) { - throw new MetastoreNotificationException( - debugString("Unable to parse the alter partition message"), e); - } - } - - public static List getEvents(NotificationEvent event, - String catalogName) { - return Lists.newArrayList(new AlterPartitionEvent(event, catalogName)); - } - - @Override - protected boolean canBeBatched(MetastoreEvent event) { - return true; - } - - @Override - protected MetastoreEvent addToBatchEvents(MetastoreEvent event) { - BatchEvent batchEvent = new BatchEvent<>(this); - Preconditions.checkState(batchEvent.canBeBatched(event)); - batchEvent.addToBatchEvents(event); - return batchEvent; - } - - @Override - protected boolean existInCache() { - return true; - } - - @Override - protected boolean canBeSkipped() { - return false; - } - - @Override - protected boolean isSupported() { - return true; - } - - @Override - protected void process() throws MetastoreNotificationException { - - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/AlterTableEvent.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/AlterTableEvent.java deleted file mode 100644 index f3ab07d011da8a..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/AlterTableEvent.java +++ /dev/null @@ -1,125 +0,0 @@ -// 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.doris.datasource.hive.event; - -import com.google.common.base.Preconditions; -import com.google.common.collect.Lists; -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.metastore.api.NotificationEvent; -import org.apache.hadoop.hive.metastore.api.Table; -import org.apache.hadoop.hive.metastore.messaging.json.JSONAlterTableMessage; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; - -import java.util.List; - -/** - * MetastoreEvent for ALTER_TABLE event type - */ -public class AlterTableEvent extends MetastoreTableEvent { - private static final Logger LOG = LogManager.getLogger(AlterTableEvent.class); - - // the table object before alter operation, as parsed from the NotificationEvent - protected Table tableBefore; - // the table object after alter operation, as parsed from the NotificationEvent - protected Table tableAfter; - // true if this alter event was due to a rename operation - protected final boolean isRename; - // true if this alter event was due to a schema change operation - protected boolean isSchemaChange = false; - - private AlterTableEvent(NotificationEvent event, String catalogName) { - super(event, catalogName); - Preconditions.checkArgument(MetastoreEventType.ALTER_TABLE.equals(getEventType())); - JSONAlterTableMessage alterTableMessage = - (JSONAlterTableMessage) MetastoreEventsProcessor.getMessageDeserializer() - .getAlterTableMessage(event.getMessage()); - try { - hmsTbl = Preconditions.checkNotNull(alterTableMessage.getTableObjBefore()); - tableAfter = Preconditions.checkNotNull(alterTableMessage.getTableObjAfter()); - tableBefore = Preconditions.checkNotNull(alterTableMessage.getTableObjBefore()); - } catch (Exception e) { - throw new MetastoreNotificationException( - debugString("Unable to parse the alter table message"), e); - } - // this is a rename event if either dbName or tblName of before and after object changed - isRename = !hmsTbl.getDbName().equalsIgnoreCase(tableAfter.getDbName()) - || !hmsTbl.getTableName().equalsIgnoreCase(tableAfter.getTableName()); - } - - public static List getEvents(NotificationEvent event, - String catalogName) { - return Lists.newArrayList(new AlterTableEvent(event, catalogName)); - } - - private boolean isSchemaChange(List before, List after) { - if (before.size() != after.size()) { - return true; - } - - if (!before.equals(after)) { - return true; - } - - return false; - } - - public boolean isSchemaChange() { - return isSchemaChange; - } - - @Override - public boolean canBeBatched(MetastoreEvent event) { - return true; - } - - @Override - protected MetastoreEvent addToBatchEvents(MetastoreEvent event) { - BatchEvent batchEvent = new BatchEvent<>(this); - Preconditions.checkState(batchEvent.canBeBatched(event)); - batchEvent.addToBatchEvents(event); - return batchEvent; - } - - @Override - protected boolean existInCache() { - - return true; - - } - - @Override - protected boolean canBeSkipped() { - return false; - } - - @Override - protected boolean isSupported() { - return true; - } - - public boolean isRename() { - return isRename; - } - - @Override - protected void process() throws MetastoreNotificationException { - - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/BatchEvent.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/BatchEvent.java deleted file mode 100644 index 7c84aa6bad1b58..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/BatchEvent.java +++ /dev/null @@ -1,158 +0,0 @@ -// 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.doris.datasource.hive.event; - -import com.google.common.base.Preconditions; -import com.google.common.collect.Lists; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; - -import java.util.List; -import java.util.stream.Collectors; - -/** - * This event represents a batch of events of type T. The batch of events is - * initialized from a single initial event called baseEvent. More events can be added - * to the batch using {@code addToBatchEvents} method. - * - * @param The type of event which is batched by this event. - */ -public class BatchEvent extends MetastoreTableEvent { - private static final Logger LOG = LogManager.getLogger(BatchEvent.class); - - private final T baseEvent; - private final List batchedEvents = Lists.newArrayList(); - - protected BatchEvent(T baseEvent) { - super(baseEvent.event, baseEvent.catalogName); - this.hmsTbl = baseEvent.hmsTbl; - this.baseEvent = baseEvent; - batchedEvents.add(baseEvent); - } - - /** - * @param event The event under consideration to be batched into this event. It can - * be added to the batch if it can be batched into the last event of the current batch. - * @return true if we can add the event to the current batch; else false. - */ - @Override - protected boolean canBeBatched(MetastoreEvent event) { - Preconditions.checkState(!batchedEvents.isEmpty()); - return batchedEvents.get(batchedEvents.size() - 1).canBeBatched(event); - } - - @Override - public MetastoreEvent addToBatchEvents(MetastoreEvent event) { - Preconditions.checkState(canBeBatched(event)); - batchedEvents.add((T) event); - return this; - } - - /** - * Return the event id of this batch event. We return the last eventId - * from this batch which is important since it is used to determined the event - * id for fetching next set of events from metastore. - */ - @Override - public long getEventId() { - Preconditions.checkState(!batchedEvents.isEmpty()); - return batchedEvents.get(batchedEvents.size() - 1).getEventId(); - } - - @Override - public int getNumberOfEvents() { - return batchedEvents.size(); - } - - public List getBatchEvents() { - return batchedEvents; - } - - /** - * Gets the event id of the first event in the batch. - */ - public long getFirstEventId() { - return batchedEvents.get(0).getEventId(); - } - - /** - * Gets the event id of the last event in the batch. - */ - public long getLastEventId() { - return batchedEvents.get(batchedEvents.size() - 1).getEventId(); - } - - /** - * Gets the event of the last event in the batch. - */ - protected MetastoreEvent getLastEvent() { - return batchedEvents.get(batchedEvents.size() - 1); - } - - @Override - protected boolean isSupported() { - return true; - } - - @Override - protected boolean existInCache() throws MetastoreNotificationException { - return baseEvent.existInCache(); - } - - /** - * The merge rule in batch event is relatively simple, because some events that don't need to be processed - * have been filtered out during the {@link MetastoreEventFactory#createBatchEvents}. mainly the following two. - * 1. Multiple ALTER_PARTITION or ALTER_TABLE only process the last one. - * 2. The first event in the batch event is ADD_PARTITION, and the ALTER events currently only takes the last one. - */ - protected List mergeBatchEvents(List eventToMerge) { - List mergedEvents = Lists.newArrayList(); - T first = eventToMerge.get(0); - T last = eventToMerge.get(eventToMerge.size() - 1); - if (first.getEventType() == MetastoreEventType.ADD_PARTITION) { - mergedEvents.add(first); - } - mergedEvents.add(last); - return mergedEvents; - } - - @Override - protected void process() throws MetastoreNotificationException { - LOG.info("Start to process batch event for {} events from {} to {}", - getNumberOfEvents(), getFirstEventId(), getLastEvent()); - if (!baseEvent.existInCache()) { - return; - } - - List eventsToProcess = batchedEvents.stream() - .filter(event -> !event.canBeSkipped()) - .collect(Collectors.toList()); - - if (eventsToProcess.isEmpty()) { - LOG.info("Ignoring events {} since they modify parameters which can be ignored", batchedEvents); - return; - } - - if (eventsToProcess.size() > 1) { - eventsToProcess = mergeBatchEvents(eventsToProcess); - } - - eventsToProcess.forEach(MetastoreEvent::process); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/CreateTableEvent.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/CreateTableEvent.java deleted file mode 100644 index 0880ce95867c46..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/CreateTableEvent.java +++ /dev/null @@ -1,60 +0,0 @@ -// 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.doris.datasource.hive.event; - -import com.google.common.base.Preconditions; -import com.google.common.collect.Lists; -import org.apache.hadoop.hive.metastore.api.NotificationEvent; -import org.apache.hadoop.hive.metastore.messaging.CreateTableMessage; - -import java.util.List; - -/** - * MetastoreEvent for CREATE_TABLE event type - */ -public class CreateTableEvent extends MetastoreTableEvent { - public static final String CREATE_TABLE_EVENT_TYPE = "CREATE_TABLE"; - - public static List getEvents(NotificationEvent event, - String catalogName) { - return Lists.newArrayList(new CreateTableEvent(event, catalogName)); - } - - private CreateTableEvent(NotificationEvent event, - String catalogName) - throws MetastoreNotificationException { - super(event, catalogName); - Preconditions.checkArgument(MetastoreEventType.CREATE_TABLE.equals(getEventType())); - Preconditions.checkNotNull(MetastoreEventType.CREATE_TABLE, debugString("Event message is null")); - CreateTableMessage createTableMessage = - MetastoreEventsProcessor.getMessageDeserializer().getCreateTableMessage(event.getMessage()); - - try { - hmsTbl = createTableMessage.getTableObj(); - } catch (Exception e) { - throw new MetastoreNotificationException( - debugString("Unable to deserialize the event message"), e); - } - } - - @Override - protected void process() throws MetastoreNotificationException { - throw new UnsupportedOperationException("Unsupported event type: " + getEventType()); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/DropPartitionEvent.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/DropPartitionEvent.java deleted file mode 100644 index 8ada42e4440fcf..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/DropPartitionEvent.java +++ /dev/null @@ -1,103 +0,0 @@ -// 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.doris.datasource.hive.event; - -import com.google.common.base.Preconditions; -import com.google.common.collect.Lists; -import org.apache.hadoop.hive.metastore.api.NotificationEvent; -import org.apache.hadoop.hive.metastore.messaging.DropPartitionMessage; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; - -import java.util.List; -import java.util.Map; - -/** - * MetastoreEvent for DROP_PARTITION event type - */ -public class DropPartitionEvent extends MetastoreTableEvent { - private static final Logger LOG = LogManager.getLogger(DropPartitionEvent.class); - public static final String EVENT_TYPE = "DROP_PARTITION"; - - private final Map droppedPartition; - - private DropPartitionEvent(NotificationEvent event, - Map droppedPartition, - String catalogName) { - super(event, catalogName); - Preconditions.checkState(getEventType().equals(MetastoreEventType.DROP_PARTITION)); - Preconditions.checkNotNull(event.getMessage()); - DropPartitionMessage dropPartitionMessage = - MetastoreEventsProcessor.getMessageDeserializer() - .getDropPartitionMessage(event.getMessage()); - try { - hmsTbl = Preconditions.checkNotNull(dropPartitionMessage.getTableObj()); - Preconditions.checkNotNull(droppedPartition); - this.droppedPartition = droppedPartition; - } catch (Exception ex) { - throw new MetastoreNotificationException( - debugString("Could not parse drop event message. "), ex); - } - } - - protected static List getEvents(NotificationEvent event, - String catalogName) { - DropPartitionMessage dropPartitionMessage = - MetastoreEventsProcessor.getMessageDeserializer() - .getDropPartitionMessage(event.getMessage()); - List dropPartitionEvents = Lists.newArrayList(); - try { - List> droppedPartitions = dropPartitionMessage.getPartitions(); - droppedPartitions.forEach(part -> - dropPartitionEvents.add(new DropPartitionEvent(event, part, catalogName))); - } catch (Exception e) { - throw new MetastoreNotificationException(e); - } - return dropPartitionEvents; - } - - @Override - public boolean canBeBatched(MetastoreEvent event) { - return true; - } - - @Override - protected MetastoreEvent addToBatchEvents(MetastoreEvent event) { - BatchEvent batchEvent = new BatchEvent<>(this); - Preconditions.checkState(batchEvent.canBeBatched(event)); - batchEvent.addToBatchEvents(event); - return batchEvent; - } - - @Override - protected boolean existInCache() { - // return cache.isPartitionPresent(getHivePartitionName()); - return true; - } - - @Override - protected boolean isSupported() { - return true; - } - - @Override - protected void process() throws MetastoreNotificationException { - - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/DropTableEvent.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/DropTableEvent.java index ff4acec1cecbc2..ab3255fc5ecbe1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/DropTableEvent.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/DropTableEvent.java @@ -18,6 +18,9 @@ package org.apache.doris.datasource.hive.event; +import org.apache.doris.catalog.Env; +import org.apache.doris.common.DdlException; + import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import org.apache.hadoop.hive.metastore.api.NotificationEvent; @@ -60,7 +63,6 @@ public static List getEvents(NotificationEvent event, @Override protected boolean existInCache() { - // return cache.isTablePresent(HiveTableName.of(dbName, tableName)); return true; } @@ -70,12 +72,16 @@ protected boolean canBeSkipped() { } protected boolean isSupported() { - // return !isResourceMappingCatalog(catalogName); return true; } @Override protected void process() throws MetastoreNotificationException { - + try { + Env.getCurrentEnv().getCatalogMgr().dropExternalTable(dbName, tableName, catalogName); + } catch (DdlException e) { + LOG.warn("DropExternalTable failed,dbName:[{}],tableName:[{}],catalogName:[{}].", dbName, tableName, + catalogName, e); + } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/EventFactory.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/EventFactory.java index 6ea2341171cde4..8fea95b6d3e03e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/EventFactory.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/EventFactory.java @@ -27,11 +27,6 @@ */ public interface EventFactory { - - // List get(NotificationEvent hmsEvent, - // CacheUpdateProcessor cacheProcessor, - // String catalogName) throws MetastoreNotificationException; - List get(NotificationEvent hmsEvent, String catalogName) throws MetastoreNotificationException; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/InsertEvent.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/InsertEvent.java deleted file mode 100644 index 1403ac39ea321f..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/InsertEvent.java +++ /dev/null @@ -1,104 +0,0 @@ -// 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.doris.datasource.hive.event; - -import com.google.common.base.Preconditions; -import com.google.common.collect.Lists; -import org.apache.hadoop.hive.metastore.api.NotificationEvent; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; - -import java.util.Collections; -import java.util.List; - -/** - * Metastore event handler for INSERT events. Handles insert events at both table and partition scopes. - * If partition is null, treat it as ALTER_TABLE event, otherwise as ALTER_PARTITION event. - */ -public class InsertEvent extends MetastoreTableEvent { - private static final Logger LOG = LogManager.getLogger(InsertEvent.class); - - // Represents the partition for this insert. Null if the table is unpartitioned. - // private final Partition insertPartition; - - private InsertEvent(NotificationEvent event, String catalogName) { - super(event, catalogName); - Preconditions.checkArgument(MetastoreEventType.INSERT.equals(getEventType())); - try { - // hmsTbl = Preconditions.checkNotNull(insertMessage.getTableObj()); - // insertPartition = insertMessage.getPtnObj(); - // if (insertPartition != null) { - // List partitionColNames = hmsTbl.getPartitionKeys().stream() - // .map(FieldSchema::getName).collect(Collectors.toList()); - // hivePartitionNames.add(HivePartitionName.of(dbName, tblName, - // FileUtils.makePartName(partitionColNames, insertPartition.getValues()))); - // } - } catch (Exception e) { - LOG.warn( - "The InsertEvent of the current hive version cannot be parsed,and there will be a corresponding " - + "Alter Event in next, InsertEvent is ignored here. {}", - e.getMessage()); - throw new MetastoreNotificationException(debugString("Unable to " - + "parse insert message"), e); - } - } - - public static List getEvents(NotificationEvent event, - String catalogName) { - try { - return Lists.newArrayList(new InsertEvent(event, catalogName)); - } catch (MetastoreNotificationException e) { - return Collections.emptyList(); - } - } - - @Override - public boolean canBeBatched(MetastoreEvent event) { - return true; - } - - @Override - protected MetastoreEvent addToBatchEvents(MetastoreEvent event) { - BatchEvent batchEvent = new BatchEvent<>(this); - Preconditions.checkState(batchEvent.canBeBatched(event)); - batchEvent.addToBatchEvents(event); - return batchEvent; - } - - @Override - protected boolean existInCache() { - return true; - } - - @Override - protected boolean isSupported() { - return true; - } - - public boolean isPartitionTbl() { - return true; - } - - @Override - protected void process() throws MetastoreNotificationException { - if (!existInCache()) { - return; - } - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEvent.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEvent.java index 96dd3d0a244713..5cc45944578285 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEvent.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEvent.java @@ -52,23 +52,8 @@ public abstract class MetastoreEvent { // Actual notificationEvent object received from Metastore protected final NotificationEvent metastoreNotificationEvent; - // The cached instance of this event that needs to be updated - // TODO: 2022/12/27 注释 - // protected final CacheUpdateProcessor cache; - protected final String catalogName; - // protected MetastoreEvent(NotificationEvent event, CacheUpdateProcessor cacheProcessor, String catalogName) { - // this.event = event; - // this.dbName = event.getDbName(); - // this.tblName = event.getTableName(); - // this.eventId = event.getEventId(); - // this.eventType = MetastoreEventType.from(event.getEventType()); - // this.metastoreNotificationEvent = event; - // this.cache = cacheProcessor; - // this.catalogName = catalogName; - // } - protected MetastoreEvent(NotificationEvent event, String catalogName) { this.event = event; this.dbName = event.getDbName(); @@ -145,7 +130,7 @@ protected boolean canBeSkipped() { /** * Whether the current version of FE supports processing of some events, some events are reserved, - * and may be processed later version, such as {@link CreateTableEvent} + * and may be processed later version. */ protected boolean isSupported() { return false; diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventFactory.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventFactory.java index 0400a77349c4a3..6497ccfc4e5a7f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventFactory.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventFactory.java @@ -18,6 +18,8 @@ package org.apache.doris.datasource.hive.event; +import org.apache.doris.datasource.HMSExternalCatalog; + import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import org.apache.hadoop.hive.metastore.api.NotificationEvent; @@ -33,58 +35,26 @@ */ public class MetastoreEventFactory implements EventFactory { private static final Logger LOG = LogManager.getLogger(MetastoreEventFactory.class); - private final List externalTables; - - public MetastoreEventFactory(List externalTables) { - this.externalTables = externalTables; - } - - public boolean needToProcess(String catalogTableName) { - return externalTables.contains(catalogTableName); - } - /** - * For an {@link AddPartitionEvent} and {@link DropPartitionEvent} drop event, - * we need to divide it into multiple events according to the number of partitions it processes. - * It is convenient for creating batch tasks to parallel processing. - */ @Override public List get(NotificationEvent event, String catalogName) { Preconditions.checkNotNull(event.getEventType()); MetastoreEventType metastoreEventType = MetastoreEventType.from(event.getEventType()); switch (metastoreEventType) { - case CREATE_TABLE: - return CreateTableEvent.getEvents(event, catalogName); - case ALTER_TABLE: - return AlterTableEvent.getEvents(event, catalogName); case DROP_TABLE: return DropTableEvent.getEvents(event, catalogName); - case ALTER_PARTITION: - return AlterPartitionEvent.getEvents(event, catalogName); - case DROP_PARTITION: - return DropPartitionEvent.getEvents(event, catalogName); - case INSERT: - return InsertEvent.getEvents(event, catalogName); default: // ignore all the unknown events by creating a IgnoredEvent return Lists.newArrayList(new IgnoredEvent(event, catalogName)); } } - List getFilteredEvents(List events, - String catalogName) { + List getMetastoreEvents(List events, HMSExternalCatalog hmsExternalCatalog) { List metastoreEvents = Lists.newArrayList(); - // Currently, the hive external table needs to be manually created in StarRocks to map with the hms table. - // Therefore, it's necessary to filter the events pulled this time from the hms instance, - // and the events of the tables that don't register in the fe MetastoreEventsProcessor need to be filtered out. for (NotificationEvent event : events) { - // String dbName = event.getDbName(); - // String tableName = event.getTableName(); - - - metastoreEvents.addAll(get(event, catalogName)); + metastoreEvents.addAll(get(event, hmsExternalCatalog.getName())); } List tobeProcessEvents = metastoreEvents.stream() @@ -92,7 +62,7 @@ List getFilteredEvents(List events, .collect(Collectors.toList()); if (tobeProcessEvents.isEmpty()) { - LOG.warn("The metastore events to process is empty on catalog {}", catalogName); + LOG.warn("The metastore events to process is empty on catalog {}", hmsExternalCatalog.getName()); return Collections.emptyList(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java index cc40ffb76b26dd..4789faaccf1f69 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java @@ -24,7 +24,6 @@ import org.apache.doris.datasource.CatalogIf; import org.apache.doris.datasource.HMSExternalCatalog; -import com.google.common.collect.Lists; import org.apache.hadoop.hive.metastore.api.NotificationEvent; import org.apache.hadoop.hive.metastore.api.NotificationEventResponse; import org.apache.hadoop.hive.metastore.messaging.MessageDeserializer; @@ -64,12 +63,12 @@ public class MetastoreEventsProcessor extends MasterDaemon { // event factory which is used to get or create MetastoreEvents private final MetastoreEventFactory metastoreEventFactory; - // [catalogName.dbName.tableName] for hive table with resource - private final List externalTables = Lists.newArrayList(); + private boolean isRunning; public MetastoreEventsProcessor() { super(MetastoreEventsProcessor.class.getName(), Config.hms_events_polling_interval_ms); - this.metastoreEventFactory = new MetastoreEventFactory(externalTables); + this.metastoreEventFactory = new MetastoreEventFactory(); + this.isRunning = false; } /** @@ -78,7 +77,7 @@ public MetastoreEventsProcessor() { */ private List getNextHMSEvents(HMSExternalCatalog hmsExternalCatalog) { LOG.error("Start to pull events on catalog [{}]", hmsExternalCatalog.getName()); - NotificationEventResponse response = hmsExternalCatalog.getNextEventResponse(hmsExternalCatalog.getName()); + NotificationEventResponse response = hmsExternalCatalog.getNextEventResponse(hmsExternalCatalog); if (response == null) { return Collections.emptyList(); @@ -115,7 +114,7 @@ private void doExecute(List events) { */ private void processEvents(List events, HMSExternalCatalog hmsExternalCatalog) { //转换过滤 - + metastoreEventFactory.getMetastoreEvents(events, hmsExternalCatalog); doExecute(events); hmsExternalCatalog.setLastSyncedEventId(events.get(events.size() - 1).getEventId()); @@ -123,6 +122,20 @@ private void processEvents(List events, HMSExternalCatalog hm @Override protected void runAfterCatalogReady() { + if (isRunning) { + LOG.warn("Last task not finished,ignore current task."); + return; + } + isRunning = true; + try { + realRun(); + } catch (Exception ex) { + LOG.error("Task failed", ex); + } + isRunning = false; + } + + private void realRun() { List catalogIds = Env.getCurrentEnv().getCatalogMgr().getCatalogIds(); for (Long catalogId : catalogIds) { CatalogIf catalog = Env.getCurrentEnv().getCatalogMgr().getCatalog(catalogId); @@ -132,7 +145,8 @@ protected void runAfterCatalogReady() { try { events = getNextHMSEvents(hmsExternalCatalog); if (!events.isEmpty()) { - LOG.error("Events size are {} on catalog [{}]", events.size(), hmsExternalCatalog.getName()); + LOG.error("Events size are {} on catalog [{}]", events.size(), + hmsExternalCatalog.getName()); processEvents(events, hmsExternalCatalog); } } catch (MetastoreNotificationFetchException e) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreTableEvent.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreTableEvent.java index 30d9f9b4b428a5..70f56bdbb06bf7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreTableEvent.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreTableEvent.java @@ -21,7 +21,6 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import org.apache.hadoop.hive.metastore.api.NotificationEvent; -import org.apache.hadoop.hive.metastore.api.Table; import java.util.List; @@ -29,22 +28,12 @@ * Base class for all the table events */ public abstract class MetastoreTableEvent extends MetastoreEvent { - // tblName from the event - protected final String tblName; - // tbl object from the Notification event, corresponds to the before tableObj in case of alter events. - protected Table hmsTbl; - - // HivePartitionName of each event to process. for unpartition table, the partition values are empty. - // protected List hivePartitionNames = Lists.newArrayList(); protected MetastoreTableEvent(NotificationEvent event, String catalogName) { super(event, catalogName); - // Preconditions.checkNotNull(dbName, "Database name cannot be null"); - tblName = Preconditions.checkNotNull(event.getTableName()); - // - // HivePartitionName hivePartitionName = new HivePartitionName(dbName, tblName, Lists.newArrayList()); - // hivePartitionNames.add(hivePartitionName); + Preconditions.checkNotNull(dbName, "Database name cannot be null"); + Preconditions.checkNotNull(tblName, "Table name cannot be null"); } /** @@ -58,11 +47,4 @@ protected MetastoreTableEvent(NotificationEvent event, String catalogName) { .add("numFiles") .add("comment") .build(); - - /** - * According to the current processing method, each event only needs to process one {@link HivePartitionName}. - */ - // protected HivePartitionName getHivePartitionName() { - // return hivePartitionNames.get(0); - // } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java index 217c0c117a981a..9254583294be30 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java @@ -949,6 +949,11 @@ public static void loadJournal(Env env, JournalEntity journal) { env.getCatalogMgr().replayRefreshExternalTable(log); break; } + case OperationType.OP_DROP_EXTERNAL_TABLE: { + final ExternalObjectLog log = (ExternalObjectLog) journal.getData(); + env.getCatalogMgr().replayDropExternalTable(log); + break; + } case OperationType.OP_INIT_EXTERNAL_TABLE: { // Do nothing. break; @@ -1624,6 +1629,10 @@ public void logRefreshExternalTable(ExternalObjectLog log) { logEdit(OperationType.OP_REFRESH_EXTERNAL_TABLE, log); } + public void logDropExternalTable(ExternalObjectLog log) { + logEdit(OperationType.OP_DROP_EXTERNAL_TABLE, log); + } + public Journal getJournal() { return this.journal; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java b/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java index 6204fc1836016a..c5889a80013d48 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java @@ -255,6 +255,8 @@ public class OperationType { public static final short OP_DROP_MTMV_TASK = 341; public static final short OP_ALTER_MTMV_TASK = 342; + public static final short OP_DROP_EXTERNAL_TABLE = 350; + public static final short OP_ALTER_USER = 400; /** From e5e415c113595110ff3f3d0caa3c323a202dd056 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Fri, 30 Dec 2022 11:32:36 +0800 Subject: [PATCH 18/25] hive event init --- .../doris/datasource/hive/event/DropTableEvent.java | 2 ++ .../hive/event/MetastoreEventsProcessor.java | 13 ++++++------- 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/DropTableEvent.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/DropTableEvent.java index ab3255fc5ecbe1..07d2cabafacece 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/DropTableEvent.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/DropTableEvent.java @@ -78,6 +78,8 @@ protected boolean isSupported() { @Override protected void process() throws MetastoreNotificationException { try { + LOG.warn("DropTable event process,catalogName:[{}],dbName:[{}],tableName:[{}]", catalogName, dbName, + tableName); Env.getCurrentEnv().getCatalogMgr().dropExternalTable(dbName, tableName, catalogName); } catch (DdlException e) { LOG.warn("DropExternalTable failed,dbName:[{}],tableName:[{}],catalogName:[{}].", dbName, tableName, diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java index 4789faaccf1f69..789a549234f018 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java @@ -101,11 +101,10 @@ private List getNextHMSEvents(HMSExternalCatalog hmsExternalC // } // } - private void doExecute(List events) { - for (NotificationEvent event : events) { - // LOG.info("event消息内容:{}", event.toString()); - LOG.error("收到hive event消息,id:为{},时间为:{},db为:{},table为:{},type为:{}", event.getEventId(), - event.getEventTime(), event.getDbName(), event.getTableName(), event.getEventType()); + private void doExecute(List events) { + for (MetastoreEvent event : events) { + LOG.error("收到hive event消息,event:[{}]", event.toString()); + event.process(); } } @@ -114,8 +113,8 @@ private void doExecute(List events) { */ private void processEvents(List events, HMSExternalCatalog hmsExternalCatalog) { //转换过滤 - metastoreEventFactory.getMetastoreEvents(events, hmsExternalCatalog); - doExecute(events); + List metastoreEvents = metastoreEventFactory.getMetastoreEvents(events, hmsExternalCatalog); + doExecute(metastoreEvents); hmsExternalCatalog.setLastSyncedEventId(events.get(events.size() - 1).getEventId()); } From 92cd48593d4bd55fb978e44bd9120d64f343f649 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Fri, 30 Dec 2022 11:59:16 +0800 Subject: [PATCH 19/25] hive event init --- .../org/apache/doris/catalog/external/HMSExternalDatabase.java | 1 + .../src/main/java/org/apache/doris/datasource/CatalogMgr.java | 3 +++ 2 files changed, 4 insertions(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalDatabase.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalDatabase.java index c23fbe637ba933..822f95f9cf4bbc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalDatabase.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalDatabase.java @@ -173,6 +173,7 @@ public void addTableForTest(HMSExternalTable tbl) { @Override public void dropTable(String tableName) { + LOG.warn("drop table [{}]", tableName); Long tableId = tableNameToId.remove(tableName); if (tableId == null) { LOG.warn("drop table [{}] failed", tableName); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java index 82ca3f0ab75179..1f0d121fd840d9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java @@ -582,9 +582,12 @@ public void dropExternalTable(String dbName, String tableName, String catalogNam // TODO: 2022/12/29 check null ? public void replayDropExternalTable(ExternalObjectLog log) { + LOG.warn("ReplayDropExternalTable,catalogId:[{}],dbId:[{}],tableId:[{}]", log.getCatalogId(), log.getDbId(), + log.getTableId()); ExternalCatalog catalog = (ExternalCatalog) idToCatalog.get(log.getCatalogId()); ExternalDatabase db = catalog.getDbForReplay(log.getDbId()); ExternalTable table = db.getTableForReplay(log.getTableId()); + db.dropTable(table.getName()); Env.getCurrentEnv().getExtMetaCacheMgr() .invalidateTableCache(catalog.getId(), db.getFullName(), table.getName()); } From 12c17cc04bf3c92bbbbe521eb28ab6fe2600d037 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Fri, 30 Dec 2022 12:12:24 +0800 Subject: [PATCH 20/25] hive event init --- .../src/main/java/org/apache/doris/journal/JournalEntity.java | 1 + 1 file changed, 1 insertion(+) diff --git a/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java b/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java index 96ae663015697f..fb8fbb5b997940 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java +++ b/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java @@ -706,6 +706,7 @@ public void readFields(DataInput in) throws IOException { break; } case OperationType.OP_REFRESH_EXTERNAL_DB: + case OperationType.OP_DROP_EXTERNAL_TABLE: case OperationType.OP_REFRESH_EXTERNAL_TABLE: { data = ExternalObjectLog.read(in); isRead = true; From f24320d257df9c52b91cf925412e01d29b475082 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Fri, 30 Dec 2022 12:26:44 +0800 Subject: [PATCH 21/25] hive event init --- fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java | 6 ++++-- .../src/main/java/org/apache/doris/common/Config.java | 6 ++++++ 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java index 5fa85986192a95..400695286b5224 100755 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Env.java @@ -1405,8 +1405,10 @@ private void startMasterOnlyDaemonThreads() { if (Config.enable_fqdn_mode) { fqdnManager.start(); } - // TODO: 2022/12/27 判断是否开启 - metastoreEventsProcessor.start(); + if (Config.enable_hms_events_incremental_sync) { + metastoreEventsProcessor.start(); + } + } // start threads that should running on all FE diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/Config.java b/fe/fe-core/src/main/java/org/apache/doris/common/Config.java index 34ae0d53d338c4..164340a8b9f533 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/Config.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/Config.java @@ -1936,6 +1936,12 @@ public class Config extends ConfigBase { @ConfField(mutable = true) public static boolean enable_func_pushdown = true; + /** + * If set to true, doris will automatically synchronize hms metadata to the cache in fe. + */ + @ConfField + public static boolean enable_hms_events_incremental_sync = false; + /** * Maximum number of events to poll in each RPC. */ From 7e997145e1c441efdfe06180b1227e5caf92513e Mon Sep 17 00:00:00 2001 From: zhangdong Date: Fri, 30 Dec 2022 15:22:59 +0800 Subject: [PATCH 22/25] hive event init --- .../apache/doris/catalog/RefreshManager.java | 21 ---------- .../catalog/external/HMSExternalDatabase.java | 2 +- .../java/org/apache/doris/common/Config.java | 6 +-- .../apache/doris/datasource/CatalogMgr.java | 3 +- .../doris/datasource/HMSExternalCatalog.java | 15 +++---- .../datasource/PooledHiveMetaStoreClient.java | 4 +- .../datasource/hive/event/DropTableEvent.java | 2 +- .../hive/event/MetastoreEventFactory.java | 3 +- .../hive/event/MetastoreEventsProcessor.java | 41 +++++++------------ 9 files changed, 29 insertions(+), 68 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/RefreshManager.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/RefreshManager.java index b224e8df678d78..b8d69299940053 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/RefreshManager.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/RefreshManager.java @@ -167,25 +167,4 @@ private void refreshExternalCtlTable(String dbName, String tableName, CatalogIf log.setTableId(table.getId()); Env.getCurrentEnv().getEditLog().logRefreshExternalTable(log); } - - private void dropExternalCtlTable(String dbName, String tableName, CatalogIf catalog) throws DdlException { - if (!(catalog instanceof ExternalCatalog)) { - throw new DdlException("Only support drop ExternalCatalog Tables"); - } - DatabaseIf db = catalog.getDbNullable(dbName); - if (db == null) { - throw new DdlException("Database " + dbName + " does not exist in catalog " + catalog.getName()); - } - - TableIf table = db.getTableNullable(tableName); - if (table == null) { - throw new DdlException("Table " + tableName + " does not exist in db " + dbName); - } - Env.getCurrentEnv().getExtMetaCacheMgr().invalidateTableCache(catalog.getId(), dbName, tableName); - ExternalObjectLog log = new ExternalObjectLog(); - log.setCatalogId(catalog.getId()); - log.setDbId(db.getId()); - log.setTableId(table.getId()); - Env.getCurrentEnv().getEditLog().logRefreshExternalTable(log); - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalDatabase.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalDatabase.java index 822f95f9cf4bbc..a1f6bcddab8f4a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalDatabase.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/external/HMSExternalDatabase.java @@ -173,7 +173,7 @@ public void addTableForTest(HMSExternalTable tbl) { @Override public void dropTable(String tableName) { - LOG.warn("drop table [{}]", tableName); + LOG.debug("drop table [{}]", tableName); Long tableId = tableNameToId.remove(tableName); if (tableId == null) { LOG.warn("drop table [{}] failed", tableName); diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/Config.java b/fe/fe-core/src/main/java/org/apache/doris/common/Config.java index 164340a8b9f533..93fbfc6e1842a0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/common/Config.java +++ b/fe/fe-core/src/main/java/org/apache/doris/common/Config.java @@ -1939,19 +1939,19 @@ public class Config extends ConfigBase { /** * If set to true, doris will automatically synchronize hms metadata to the cache in fe. */ - @ConfField + @ConfField(masterOnly = true) public static boolean enable_hms_events_incremental_sync = false; /** * Maximum number of events to poll in each RPC. */ - @ConfField(mutable = true) + @ConfField(mutable = true, masterOnly = true) public static int hms_events_batch_size_per_rpc = 500; /** * HMS polling interval in milliseconds. */ - @ConfField + @ConfField(masterOnly = true) public static int hms_events_polling_interval_ms = 20000; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java index 1f0d121fd840d9..05729cb36872ed 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogMgr.java @@ -580,9 +580,8 @@ public void dropExternalTable(String dbName, String tableName, String catalogNam Env.getCurrentEnv().getEditLog().logDropExternalTable(log); } - // TODO: 2022/12/29 check null ? public void replayDropExternalTable(ExternalObjectLog log) { - LOG.warn("ReplayDropExternalTable,catalogId:[{}],dbId:[{}],tableId:[{}]", log.getCatalogId(), log.getDbId(), + LOG.debug("ReplayDropExternalTable,catalogId:[{}],dbId:[{}],tableId:[{}]", log.getCatalogId(), log.getDbId(), log.getTableId()); ExternalCatalog catalog = (ExternalCatalog) idToCatalog.get(log.getCatalogId()); ExternalDatabase db = catalog.getDbForReplay(log.getDbId()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java index b024550bb80dd4..ffcd724f5655be 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java @@ -67,8 +67,6 @@ public HMSExternalCatalog( props.putAll(HMSResource.getPropertiesFromDLF()); } catalogProperty = new CatalogProperty(resource, props); - // setLastSyncedEventId(getCurrentEventId()); - // LOG.error("new HMSExternalCatalog(),lastSyncedEventId = {}", lastSyncedEventId); } public String getHiveMetastoreUris() { @@ -193,7 +191,7 @@ public NotificationEventResponse getNextEventResponse(HMSExternalCatalog hmsExte if (lastSyncedEventId <= 0) { lastSyncedEventId = getCurrentEventId(); refreshCatalog(hmsExternalCatalog); - LOG.error( + LOG.info( "First pulling events on catalog [{}],refreshCatalog and init lastSyncedEventId," + "lastSyncedEventId is [{}]", hmsExternalCatalog.getName(), lastSyncedEventId); @@ -201,9 +199,10 @@ public NotificationEventResponse getNextEventResponse(HMSExternalCatalog hmsExte } long currentEventId = getCurrentEventId(); - LOG.error("本次getNextEventResponse的currentEventId为{},lastSyncedEventId为{}", currentEventId, lastSyncedEventId); + LOG.debug("Catalog [{}] getNextEventResponse的currentEventId is {},lastSyncedEventId is {}", + hmsExternalCatalog.getName(), currentEventId, lastSyncedEventId); if (currentEventId == lastSyncedEventId) { - LOG.error("Event id not updated when pulling events on catalog [{}]", hmsExternalCatalog.getName()); + LOG.info("Event id not updated when pulling events on catalog [{}]", hmsExternalCatalog.getName()); return null; } return client.getNextNotification(lastSyncedEventId, Config.hms_events_batch_size_per_rpc, null); @@ -218,13 +217,9 @@ private void refreshCatalog(HMSExternalCatalog hmsExternalCatalog) { public long getCurrentEventId() { makeSureInitialized(); - if (client == null) { - LOG.error("client为空===================="); - return -1; - } CurrentNotificationEventId currentNotificationEventId = client.getCurrentNotificationEventId(); if (currentNotificationEventId == null) { - LOG.error("获取currentNotificationEventId为null"); + LOG.warn("Get currentNotificationEventId is null"); return -1; } return currentNotificationEventId.getEventId(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/PooledHiveMetaStoreClient.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/PooledHiveMetaStoreClient.java index ce248ed7ccefae..008253c45003b8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/PooledHiveMetaStoreClient.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/PooledHiveMetaStoreClient.java @@ -152,7 +152,7 @@ public CurrentNotificationEventId getCurrentNotificationEventId() { try (CachedClient client = getClient()) { return client.client.getCurrentNotificationEventId(); } catch (Exception e) { - LOG.error("Failed to fetch current notification event id", e); + LOG.warn("Failed to fetch current notification event id", e); throw new MetastoreNotificationFetchException( "Failed to get current notification event id. msg: " + e.getMessage()); } @@ -165,7 +165,7 @@ public NotificationEventResponse getNextNotification(long lastEventId, try (CachedClient client = getClient()) { return client.client.getNextNotification(lastEventId, maxEvents, filter); } catch (Exception e) { - LOG.error("Failed to get next notification based on last event id {}", lastEventId, e); + LOG.warn("Failed to get next notification based on last event id {}", lastEventId, e); throw new MetastoreNotificationFetchException( "Failed to get next notification based on last event id: " + lastEventId + ". msg: " + e .getMessage()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/DropTableEvent.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/DropTableEvent.java index 07d2cabafacece..8647e47b7864c9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/DropTableEvent.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/DropTableEvent.java @@ -78,7 +78,7 @@ protected boolean isSupported() { @Override protected void process() throws MetastoreNotificationException { try { - LOG.warn("DropTable event process,catalogName:[{}],dbName:[{}],tableName:[{}]", catalogName, dbName, + LOG.info("DropTable event process,catalogName:[{}],dbName:[{}],tableName:[{}]", catalogName, dbName, tableName); Env.getCurrentEnv().getCatalogMgr().dropExternalTable(dbName, tableName, catalogName); } catch (DdlException e) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventFactory.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventFactory.java index 6497ccfc4e5a7f..a9073323388a2c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventFactory.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventFactory.java @@ -62,7 +62,7 @@ List getMetastoreEvents(List events, HMSExter .collect(Collectors.toList()); if (tobeProcessEvents.isEmpty()) { - LOG.warn("The metastore events to process is empty on catalog {}", hmsExternalCatalog.getName()); + LOG.info("The metastore events to process is empty on catalog {}", hmsExternalCatalog.getName()); return Collections.emptyList(); } @@ -75,6 +75,7 @@ List getMetastoreEvents(List events, HMSExter * For a partition, it is meaningless to process any events before the drop partition. */ List createBatchEvents(List events) { + // now do nothing return events; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java index 789a549234f018..1ff3bd98b24936 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventsProcessor.java @@ -76,7 +76,7 @@ public MetastoreEventsProcessor() { * {@link Config#hms_events_batch_size_per_rpc} */ private List getNextHMSEvents(HMSExternalCatalog hmsExternalCatalog) { - LOG.error("Start to pull events on catalog [{}]", hmsExternalCatalog.getName()); + LOG.debug("Start to pull events on catalog [{}]", hmsExternalCatalog.getName()); NotificationEventResponse response = hmsExternalCatalog.getNextEventResponse(hmsExternalCatalog); if (response == null) { @@ -85,26 +85,14 @@ private List getNextHMSEvents(HMSExternalCatalog hmsExternalC return response.getEvents(); } - - // private void doExecute(List events) { - // for (MetastoreEvent event : events) { - // try { - // event.process(); - // } catch (Exception e) { - // if (event instanceof BatchEvent) { - // cacheProcessor.setLastSyncedEventId(((BatchEvent) event).getFirstEventId() - 1); - // } else { - // cacheProcessor.setLastSyncedEventId(event.getEventId() - 1); - // } - // throw e; - // } - // } - // } - - private void doExecute(List events) { + private void doExecute(List events, HMSExternalCatalog hmsExternalCatalog) { for (MetastoreEvent event : events) { - LOG.error("收到hive event消息,event:[{}]", event.toString()); - event.process(); + try { + event.process(); + } catch (Exception e) { + hmsExternalCatalog.setLastSyncedEventId(event.getEventId() - 1); + throw e; + } } } @@ -112,10 +100,9 @@ private void doExecute(List events) { * Process the given list of notification events. Useful for tests which provide a list of events */ private void processEvents(List events, HMSExternalCatalog hmsExternalCatalog) { - //转换过滤 + //transfer List metastoreEvents = metastoreEventFactory.getMetastoreEvents(events, hmsExternalCatalog); - doExecute(metastoreEvents); - + doExecute(metastoreEvents, hmsExternalCatalog); hmsExternalCatalog.setLastSyncedEventId(events.get(events.size() - 1).getEventId()); } @@ -129,7 +116,7 @@ protected void runAfterCatalogReady() { try { realRun(); } catch (Exception ex) { - LOG.error("Task failed", ex); + LOG.warn("Task failed", ex); } isRunning = false; } @@ -144,14 +131,14 @@ private void realRun() { try { events = getNextHMSEvents(hmsExternalCatalog); if (!events.isEmpty()) { - LOG.error("Events size are {} on catalog [{}]", events.size(), + LOG.info("Events size are {} on catalog [{}]", events.size(), hmsExternalCatalog.getName()); processEvents(events, hmsExternalCatalog); } } catch (MetastoreNotificationFetchException e) { - LOG.error("Failed to fetch hms events on {}. msg: ", hmsExternalCatalog.getName(), e); + LOG.warn("Failed to fetch hms events on {}. msg: ", hmsExternalCatalog.getName(), e); } catch (Exception ex) { - LOG.error("Failed to process hive metastore [{}] events .", + LOG.warn("Failed to process hive metastore [{}] events .", hmsExternalCatalog.getName(), ex); } } From fd819993dc6728957e4de29e29173fff36db6777 Mon Sep 17 00:00:00 2001 From: zhangdong Date: Fri, 30 Dec 2022 18:04:49 +0800 Subject: [PATCH 23/25] lastSyncedEventId init to -1 --- .../main/java/org/apache/doris/datasource/ExternalCatalog.java | 3 +++ .../java/org/apache/doris/datasource/HMSExternalCatalog.java | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java index 2d68d28feebbe4..e730fb6bc07014 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java @@ -295,6 +295,9 @@ public void gsonPostProcess() throws IOException { db.setTableExtCatalog(this); } objectCreated = false; + if (this instanceof HMSExternalCatalog) { + ((HMSExternalCatalog) this).setLastSyncedEventId(-1L); + } } public void addDatabaseForTest(ExternalDatabase db) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java index ffcd724f5655be..ab3ea6bf267b0d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java @@ -188,7 +188,7 @@ public void setLastSyncedEventId(long lastSyncedEventId) { public NotificationEventResponse getNextEventResponse(HMSExternalCatalog hmsExternalCatalog) throws MetastoreNotificationFetchException { makeSureInitialized(); - if (lastSyncedEventId <= 0) { + if (lastSyncedEventId < 0) { lastSyncedEventId = getCurrentEventId(); refreshCatalog(hmsExternalCatalog); LOG.info( From f2da5f23045b215aa3eb718f4b40f518307bccba Mon Sep 17 00:00:00 2001 From: zhangdong Date: Tue, 3 Jan 2023 09:35:21 +0800 Subject: [PATCH 24/25] change getCurrentEventId from 'public' to 'private' --- .../java/org/apache/doris/datasource/HMSExternalCatalog.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java index ab3ea6bf267b0d..f48e58d3775b01 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java @@ -215,7 +215,7 @@ private void refreshCatalog(HMSExternalCatalog hmsExternalCatalog) { Env.getCurrentEnv().getCatalogMgr().refreshCatalog(log); } - public long getCurrentEventId() { + private long getCurrentEventId() { makeSureInitialized(); CurrentNotificationEventId currentNotificationEventId = client.getCurrentNotificationEventId(); if (currentNotificationEventId == null) { From 4c391d64ae69e1fb554f8240dc7edf241b440a1a Mon Sep 17 00:00:00 2001 From: zhangdong Date: Tue, 3 Jan 2023 11:06:19 +0800 Subject: [PATCH 25/25] change method name and log --- .../java/org/apache/doris/datasource/HMSExternalCatalog.java | 2 +- .../org/apache/doris/datasource/hive/event/EventFactory.java | 2 +- .../doris/datasource/hive/event/MetastoreEventFactory.java | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java index f48e58d3775b01..ec94867fc103fe 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/HMSExternalCatalog.java @@ -199,7 +199,7 @@ public NotificationEventResponse getNextEventResponse(HMSExternalCatalog hmsExte } long currentEventId = getCurrentEventId(); - LOG.debug("Catalog [{}] getNextEventResponse的currentEventId is {},lastSyncedEventId is {}", + LOG.debug("Catalog [{}] getNextEventResponse, currentEventId is {},lastSyncedEventId is {}", hmsExternalCatalog.getName(), currentEventId, lastSyncedEventId); if (currentEventId == lastSyncedEventId) { LOG.info("Event id not updated when pulling events on catalog [{}]", hmsExternalCatalog.getName()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/EventFactory.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/EventFactory.java index 8fea95b6d3e03e..333687e2ab384d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/EventFactory.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/EventFactory.java @@ -27,6 +27,6 @@ */ public interface EventFactory { - List get(NotificationEvent hmsEvent, + List transferNotificationEventToMetastoreEvents(NotificationEvent hmsEvent, String catalogName) throws MetastoreNotificationException; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventFactory.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventFactory.java index a9073323388a2c..2719158c8e9836 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventFactory.java +++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/MetastoreEventFactory.java @@ -37,7 +37,7 @@ public class MetastoreEventFactory implements EventFactory { private static final Logger LOG = LogManager.getLogger(MetastoreEventFactory.class); @Override - public List get(NotificationEvent event, + public List transferNotificationEventToMetastoreEvents(NotificationEvent event, String catalogName) { Preconditions.checkNotNull(event.getEventType()); MetastoreEventType metastoreEventType = MetastoreEventType.from(event.getEventType()); @@ -54,7 +54,7 @@ List getMetastoreEvents(List events, HMSExter List metastoreEvents = Lists.newArrayList(); for (NotificationEvent event : events) { - metastoreEvents.addAll(get(event, hmsExternalCatalog.getName())); + metastoreEvents.addAll(transferNotificationEventToMetastoreEvents(event, hmsExternalCatalog.getName())); } List tobeProcessEvents = metastoreEvents.stream()