You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by mo...@apache.org on 2023/04/14 13:19:26 UTC

[doris] 02/04: [Fix](multi-catalog) add handler for hms INSERT EVENT. (#17933)

This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 2bb84fc4f4b0e47e0bc7a01bebb68fc2bf50c230
Author: Xiangyu Wang <du...@gmail.com>
AuthorDate: Fri Mar 24 10:17:47 2023 +0800

    [Fix](multi-catalog) add handler for hms INSERT EVENT. (#17933)
    
    When we use a hive client to submit a `INSERT INTO TBL SELECT * FROM ...` or `INSERT INTO TBL VALUES ...`
    sql and the table is non-partitioned table, the hms will generate an insert event. The insert stmt may changed the
    hdfs file distribution of this table, but currently we do not handle this, so the file cache of this table may be inaccurate.
---
 .../doris/datasource/hive/event/InsertEvent.java   | 75 ++++++++++++++++++++++
 .../hive/event/MetastoreEventFactory.java          |  2 +
 2 files changed, 77 insertions(+)

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 0000000000..cf4ba1d5b0
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/hive/event/InsertEvent.java
@@ -0,0 +1,75 @@
+// 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.DdlException;
+
+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.Table;
+import org.apache.hadoop.hive.metastore.messaging.InsertMessage;
+
+import java.util.List;
+
+/**
+ * MetastoreEvent for INSERT event type
+ */
+public class InsertEvent extends MetastoreTableEvent {
+    private final Table hmsTbl;
+
+    private InsertEvent(NotificationEvent event, String catalogName) {
+        super(event, catalogName);
+        Preconditions.checkArgument(getEventType().equals(MetastoreEventType.INSERT));
+        Preconditions
+                .checkNotNull(event.getMessage(), debugString("Event message is null"));
+        try {
+            InsertMessage insertMessage =
+                    MetastoreEventsProcessor.getMessageDeserializer(event.getMessageFormat())
+                            .getInsertMessage(event.getMessage());
+            hmsTbl = Preconditions.checkNotNull(insertMessage.getTableObj());
+        } catch (Exception ex) {
+            throw new MetastoreNotificationException(ex);
+        }
+    }
+
+    protected static List<MetastoreEvent> getEvents(NotificationEvent event, String catalogName) {
+        return Lists.newArrayList(new InsertEvent(event, catalogName));
+    }
+
+    @Override
+    protected void process() throws MetastoreNotificationException {
+        try {
+            infoLog("catalogName:[{}],dbName:[{}],tableName:[{}]", catalogName, dbName, tblName);
+            /***
+             *  Only when we use hive client to execute a `INSERT INTO TBL SELECT * ...` or `INSERT INTO TBL ...` sql
+             *  to a non-partitioned table then the hms will generate an insert event, and there is not
+             *  any partition event occurs, but the file cache may has been changed, so we need handle this.
+             *  Currently {@link org.apache.doris.datasource.CatalogMgr#refreshExternalTable} do not invalidate
+             *  the file cache of this table,
+             *  but <a href="https://github.com/apache/doris/pull/17932">this PR</a> has fixed it.
+             */
+            Env.getCurrentEnv().getCatalogMgr().refreshExternalTable(dbName, tblName, catalogName);
+        } catch (DdlException e) {
+            throw new MetastoreNotificationException(
+                    debugString("Failed to process event"));
+        }
+    }
+}
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 ce96ce62e1..3ab2a7e030 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
@@ -58,6 +58,8 @@ public class MetastoreEventFactory implements EventFactory {
                 return DropPartitionEvent.getEvents(event, catalogName);
             case ALTER_PARTITION:
                 return AlterPartitionEvent.getEvents(event, catalogName);
+            case INSERT:
+                return InsertEvent.getEvents(event, catalogName);
             default:
                 // ignore all the unknown events by creating a IgnoredEvent
                 return IgnoredEvent.getEvents(event, catalogName);


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org