You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by GitBox <gi...@apache.org> on 2022/06/14 14:48:55 UTC

[GitHub] [incubator-doris] morningman commented on a diff in pull request #10033: [feature-wip](multi-catalog) Catalog operation syntax

morningman commented on code in PR #10033:
URL: https://github.com/apache/incubator-doris/pull/10033#discussion_r896886132


##########
fe/fe-core/src/main/cup/sql_parser.cup:
##########
@@ -2692,6 +2712,16 @@ show_param ::=
     {:
         RESULT = new ShowDbStmt(parser.wild, parser.where);
     :}
+    /* Catalog */
+    | KW_CATALOG

Review Comment:
   Add new keywords `KW_CATALOGS`



##########
fe/fe-core/src/main/java/org/apache/doris/datasource/DataSourceLog.java:
##########
@@ -0,0 +1,62 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.datasource;
+
+import org.apache.doris.common.io.Text;
+import org.apache.doris.common.io.Writable;
+import org.apache.doris.persist.gson.GsonUtils;
+
+import com.google.gson.annotations.SerializedName;
+import lombok.Data;
+import lombok.Getter;
+import lombok.NoArgsConstructor;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Map;
+
+/**
+ * A union metadata log for all the datasource operator include create,drop and alter.
+ */
+@NoArgsConstructor
+@Getter
+@Data
+public class DataSourceLog implements Writable {
+    @SerializedName(value = "dsName")
+    private String dsName;

Review Comment:
   change to `catalogName`, so that we don't need to modify the persist content once we change datasource to catalog



##########
fe/fe-core/src/main/cup/sql_parser.cup:
##########
@@ -277,7 +277,8 @@ terminal String KW_ADD, KW_ADMIN, KW_AFTER, KW_AGGREGATE, KW_ALIAS, KW_ALL, KW_A
     KW_VALUE, KW_VALUES, KW_VARCHAR, KW_VARIABLES, KW_VERBOSE, KW_VIEW,
     KW_WARNINGS, KW_WEEK, KW_WHEN, KW_WHITELIST, KW_WHERE, KW_WITH, KW_WORK, KW_WRITE,
     KW_YEAR,
-    KW_NOT_NULL;
+    KW_NOT_NULL,
+    KW_CATALOG;

Review Comment:
   Add the `KW_CATALOG` to the `keyword ::=` field in this file.



##########
fe/fe-core/src/main/java/org/apache/doris/datasource/DataSourceLog.java:
##########
@@ -0,0 +1,62 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.datasource;
+
+import org.apache.doris.common.io.Text;
+import org.apache.doris.common.io.Writable;
+import org.apache.doris.persist.gson.GsonUtils;
+
+import com.google.gson.annotations.SerializedName;
+import lombok.Data;
+import lombok.Getter;
+import lombok.NoArgsConstructor;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Map;
+
+/**
+ * A union metadata log for all the datasource operator include create,drop and alter.
+ */
+@NoArgsConstructor
+@Getter
+@Data
+public class DataSourceLog implements Writable {
+    @SerializedName(value = "dsName")
+    private String dsName;
+
+    @SerializedName(value = "props")
+    private Map<String, String> props;
+
+    @SerializedName(value = "newDsName")
+    private String newDsName;

Review Comment:
   newCatalogName



##########
fe/fe-core/src/main/java/org/apache/doris/analysis/ShowCatalogStmt.java:
##########
@@ -0,0 +1,93 @@
+// 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.analysis;
+
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.ScalarType;
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.common.Config;
+import org.apache.doris.common.UserException;
+import org.apache.doris.qe.ShowResultSetMetaData;
+
+/**
+ * Statement for show all catalog or desc the specific catalog.
+ */
+public class ShowCatalogStmt extends ShowStmt {
+    private static final ShowResultSetMetaData META_DATA_ALL =
+            ShowResultSetMetaData.builder()
+                    .addColumn(new Column("CatalogName", ScalarType.createVarchar(30)))
+                    .addColumn(new Column("Type", ScalarType.createVarchar(30)))
+                    .build();
+
+    private static final ShowResultSetMetaData META_DATA_SPECIFIC =
+            ShowResultSetMetaData.builder()
+                    .addColumn(new Column("CatalogName", ScalarType.createVarchar(30)))
+                    .addColumn(new Column("Type", ScalarType.createVarchar(30)))
+                    .addColumn(new Column("Props", ScalarType.createVarchar(8092)))
+                    .build();

Review Comment:
   When specify a certain catalog, how about show result like:
   |Key|Value|
   |---|---|
   |type | hms |
   | key1 |value1|
   |key2| value2|
   
   That could be more easy to read?



##########
fe/fe-core/src/main/java/org/apache/doris/analysis/CreateCatalogStmt.java:
##########
@@ -0,0 +1,98 @@
+// 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.analysis;
+
+import org.apache.doris.catalog.Catalog;
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.common.Config;
+import org.apache.doris.common.ErrorCode;
+import org.apache.doris.common.ErrorReport;
+import org.apache.doris.common.FeNameFormat;
+import org.apache.doris.common.UserException;
+import org.apache.doris.common.util.PrintableMap;
+import org.apache.doris.datasource.InternalDataSource;
+import org.apache.doris.mysql.privilege.PrivPredicate;
+import org.apache.doris.qe.ConnectContext;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Statement for create a new catalog.
+ */
+public class CreateCatalogStmt extends DdlStmt {
+    private final boolean ifNotExists;
+    private final String catalogName;
+    private final Map<String, String> properties;
+
+    /**
+     * Statement for create a new catalog.
+     */
+    public CreateCatalogStmt(boolean ifNotExists, String catalogName, Map<String, String> properties) {
+        this.ifNotExists = ifNotExists;
+        this.catalogName = catalogName;
+        this.properties = properties == null ? new HashMap<>() : properties;
+    }
+
+    public String getCatalogName() {
+        return catalogName;
+    }
+
+    public Map<String, String> getProperties() {
+        return properties;
+    }
+
+    public boolean isSetIfNotExists() {
+        return ifNotExists;
+    }
+
+    @Override
+    public void analyze(Analyzer analyzer) throws UserException {
+        super.analyze(analyzer);
+        if (!Config.enable_multi_catalog) {
+            throw new AnalysisException("The multi-catalog feature is still in experiment, and you can enable it "
+                    + "manually by set fe configuration named `enable_multi_catalog` to be ture.");
+        }
+        if (catalogName.equals(InternalDataSource.INTERNAL_DS_NAME)) {
+            throw new AnalysisException("Internal catalog name can't be create.");
+        }
+        FeNameFormat.checkCatalogName(catalogName);

Review Comment:
   You can just use `checkCommonName`



##########
fe/fe-core/src/main/java/org/apache/doris/datasource/DataSourceMgr.java:
##########
@@ -52,103 +66,180 @@ public DataSourceMgr() {
 
     private void initInternalDataSource() {
         internalDataSource = new InternalDataSource();
-        idToDataSource.put(internalDataSource.getId(), internalDataSource);
         nameToDataSource.put(internalDataSource.getName(), internalDataSource);
     }
 
-    private void registerNewDataSource(ExternalDataSource ds) {
-        // TODO
-    }
-
     public InternalDataSource getInternalDataSource() {
         return internalDataSource;
     }
 
+    private void writeLock() {
+        lock.writeLock().lock();
+    }
+
+    private void writeUnlock() {
+        lock.writeLock().unlock();
+    }
+
+    private void readLock() {
+        lock.readLock().lock();
+    }
+
+    private void readUnlock() {
+        lock.readLock().unlock();
+    }
+
     /**
-     * get data source by id.
-     *
-     * @param id
-     * @param e
-     * @param <E>
-     * @return
-     * @throws E
+     * Create and hold the datasource instance and write the meta log.
      */
-    public <E extends MetaNotFoundException> DataSourceIf getDataSourceOrException(long id,
-            java.util.function.Function<Long, E> e) throws E {
-        DataSourceIf ds = idToDataSource.get(id);
-        if (ds == null) {
-            throw e.apply(id);
+    public void createDatasource(CreateCatalogStmt stmt) throws UserException {
+        if (stmt.isSetIfNotExists() && nameToDataSource.containsKey(stmt.getCatalogName())) {
+            LOG.warn("Datasource {} is already exist.", stmt.getCatalogName());
+            return;
         }
-        return ds;
+        DataSourceLog log = DataSourceFactory.constructorDatasourceLog(stmt);
+        replyCreateDs(log);
+        Catalog.getCurrentCatalog().getEditLog().logDatasourceLog(OperationType.OP_CREATE_DS, log);
     }
 
     /**
-     * get data source by name.
-     *
-     * @param name
-     * @param e
-     * @param <E>
-     * @return
-     * @throws E
+     * Remove the datasource instance by name and write the meta log.
      */
-    public <E extends MetaNotFoundException> DataSourceIf getDataSourceOrException(String name,
-            java.util.function.Function<String, E> e) throws E {
-        DataSourceIf ds = nameToDataSource.get(name);
-        if (ds == null) {
-            throw e.apply(name);
+    public void dropDatasource(DropCatalogStmt stmt) throws UserException {
+        if (stmt.isSetIfExists() && !nameToDataSource.containsKey(stmt.getCatalogName())) {
+            LOG.warn("Non datasource {} is found.", stmt.getCatalogName());
+            return;
         }
-        return ds;
+        DataSourceLog log = DataSourceFactory.constructorDatasourceLog(stmt);
+        replyDropDs(log);

Review Comment:
   ```suggestion
           replayDropDs(log);
   ```



##########
fe/fe-core/src/main/java/org/apache/doris/datasource/DataSourceMgr.java:
##########
@@ -52,103 +66,180 @@ public DataSourceMgr() {
 
     private void initInternalDataSource() {
         internalDataSource = new InternalDataSource();
-        idToDataSource.put(internalDataSource.getId(), internalDataSource);
         nameToDataSource.put(internalDataSource.getName(), internalDataSource);
     }
 
-    private void registerNewDataSource(ExternalDataSource ds) {
-        // TODO
-    }
-
     public InternalDataSource getInternalDataSource() {
         return internalDataSource;
     }
 
+    private void writeLock() {
+        lock.writeLock().lock();
+    }
+
+    private void writeUnlock() {
+        lock.writeLock().unlock();
+    }
+
+    private void readLock() {
+        lock.readLock().lock();
+    }
+
+    private void readUnlock() {
+        lock.readLock().unlock();
+    }
+
     /**
-     * get data source by id.
-     *
-     * @param id
-     * @param e
-     * @param <E>
-     * @return
-     * @throws E
+     * Create and hold the datasource instance and write the meta log.
      */
-    public <E extends MetaNotFoundException> DataSourceIf getDataSourceOrException(long id,
-            java.util.function.Function<Long, E> e) throws E {
-        DataSourceIf ds = idToDataSource.get(id);
-        if (ds == null) {
-            throw e.apply(id);
+    public void createDatasource(CreateCatalogStmt stmt) throws UserException {
+        if (stmt.isSetIfNotExists() && nameToDataSource.containsKey(stmt.getCatalogName())) {
+            LOG.warn("Datasource {} is already exist.", stmt.getCatalogName());
+            return;
         }
-        return ds;
+        DataSourceLog log = DataSourceFactory.constructorDatasourceLog(stmt);
+        replyCreateDs(log);

Review Comment:
   ```suggestion
           replayCreateDs(log);
   ```



##########
fe/fe-core/src/main/java/org/apache/doris/datasource/DataSourceMgr.java:
##########
@@ -52,103 +66,180 @@ public DataSourceMgr() {
 
     private void initInternalDataSource() {
         internalDataSource = new InternalDataSource();
-        idToDataSource.put(internalDataSource.getId(), internalDataSource);
         nameToDataSource.put(internalDataSource.getName(), internalDataSource);
     }
 
-    private void registerNewDataSource(ExternalDataSource ds) {
-        // TODO
-    }
-
     public InternalDataSource getInternalDataSource() {
         return internalDataSource;
     }
 
+    private void writeLock() {
+        lock.writeLock().lock();
+    }
+
+    private void writeUnlock() {
+        lock.writeLock().unlock();
+    }
+
+    private void readLock() {
+        lock.readLock().lock();
+    }
+
+    private void readUnlock() {
+        lock.readLock().unlock();
+    }
+
     /**
-     * get data source by id.
-     *
-     * @param id
-     * @param e
-     * @param <E>
-     * @return
-     * @throws E
+     * Create and hold the datasource instance and write the meta log.
      */
-    public <E extends MetaNotFoundException> DataSourceIf getDataSourceOrException(long id,
-            java.util.function.Function<Long, E> e) throws E {
-        DataSourceIf ds = idToDataSource.get(id);
-        if (ds == null) {
-            throw e.apply(id);
+    public void createDatasource(CreateCatalogStmt stmt) throws UserException {
+        if (stmt.isSetIfNotExists() && nameToDataSource.containsKey(stmt.getCatalogName())) {
+            LOG.warn("Datasource {} is already exist.", stmt.getCatalogName());
+            return;
         }
-        return ds;
+        DataSourceLog log = DataSourceFactory.constructorDatasourceLog(stmt);
+        replyCreateDs(log);
+        Catalog.getCurrentCatalog().getEditLog().logDatasourceLog(OperationType.OP_CREATE_DS, log);
     }
 
     /**
-     * get data source by name.
-     *
-     * @param name
-     * @param e
-     * @param <E>
-     * @return
-     * @throws E
+     * Remove the datasource instance by name and write the meta log.
      */
-    public <E extends MetaNotFoundException> DataSourceIf getDataSourceOrException(String name,
-            java.util.function.Function<String, E> e) throws E {
-        DataSourceIf ds = nameToDataSource.get(name);
-        if (ds == null) {
-            throw e.apply(name);
+    public void dropDatasource(DropCatalogStmt stmt) throws UserException {
+        if (stmt.isSetIfExists() && !nameToDataSource.containsKey(stmt.getCatalogName())) {
+            LOG.warn("Non datasource {} is found.", stmt.getCatalogName());
+            return;
         }
-        return ds;
+        DataSourceLog log = DataSourceFactory.constructorDatasourceLog(stmt);
+        replyDropDs(log);
+        Catalog.getCurrentCatalog().getEditLog().logDatasourceLog(OperationType.OP_DROP_DS, log);
     }
 
-    public boolean hasDataSource(String name) {
-        return nameToDataSource.containsKey(name);
+    /**
+     * Modify the datasource name into a new one and write the meta log.
+     */
+    public void alterDatasourceName(AlterCatalogNameStmt stmt) throws UserException {
+        DataSourceLog log = DataSourceFactory.constructorDatasourceLog(stmt);
+        replyAlterDsName(log);
+        Catalog.getCurrentCatalog().getEditLog().logDatasourceLog(OperationType.OP_ALTER_DS_NAME, log);
     }
 
-    @Override
-    public void write(DataOutput out) throws IOException {
-        if (Config.disable_cluster_feature) {
-            return;
+    /**
+     * Modify the datasource property and write the meta log.
+     */
+    public void alterDatasourceProps(AlterCatalogPropertyStmt stmt) throws UserException {
+        DataSourceLog log = DataSourceFactory.constructorDatasourceLog(stmt);
+        replyAlterDsProps(log);
+        Catalog.getCurrentCatalog().getEditLog().logDatasourceLog(OperationType.OP_ALTER_DS_PROPS, log);
+    }
+
+    /**
+     * List all datasource or get the special datasource with a name.
+     */
+    public ShowResultSet showDatasource(ShowCatalogStmt showStmt) throws AnalysisException {
+        List<List<String>> rows = Lists.newArrayList();
+        readLock();
+        try {
+            if (showStmt.getDsName() == null) {
+                for (DataSourceIf ds : nameToDataSource.values()) {
+                    List<String> row = Lists.newArrayList();
+                    row.add(ds.getName());
+                    row.add(ds.getType());
+                    rows.add(row);
+                }
+            } else {
+                if (!nameToDataSource.containsKey(showStmt.getDsName())) {
+                    throw new AnalysisException("No datasource found with name: " + showStmt.getDsName());
+                }
+                DataSourceIf ds = nameToDataSource.get(showStmt.getDsName());
+                List<String> row = Lists.newArrayList();
+                row.add(ds.getName());
+                row.add(ds.getType());
+                row.add(ds.getPropertyString());
+                rows.add(row);
+            }
+        } finally {
+            readUnlock();
         }
-        Preconditions.checkState(false, "Do not call this until multi catalog feature is ready");
-        int size = idToDataSource.size();
-        if (idToDataSource.get(InternalDataSource.INTERNAL_DS_ID) != null) {
-            // No need to persis internal data source
-            size -= 1;
+
+        return new ShowResultSet(showStmt.getMetaData(), rows);
+    }
+
+    /**
+     * Reply for create ds event.
+     */
+    public void replyCreateDs(DataSourceLog log) throws UserException {
+        writeLock();
+        try {
+            if (nameToDataSource.containsKey(log.getDsName())) {
+                throw new DdlException("Datasource had already exist with name: " + log.getDsName());
+            }
+            DataSourceIf ds = DataSourceFactory.constructorFromLog(log);
+            nameToDataSource.put(ds.getName(), ds);
+        } finally {
+            writeUnlock();
         }
-        out.writeInt(size);
-        for (DataSourceIf ds : idToDataSource.values()) {
-            if (ds.getId() == InternalDataSource.INTERNAL_DS_ID) {
-                continue;
+    }
+
+    /**
+     * Reply for drop ds event.
+     */
+    public void replyDropDs(DataSourceLog log) throws UserException {
+        writeLock();
+        try {
+            if (!nameToDataSource.containsKey(log.getDsName())) {
+                throw new DdlException("No datasource found with name: " + log.getDsName());
             }
-            ExternalDataSource extDs = (ExternalDataSource) ds;
-            extDs.write(out);
+            nameToDataSource.remove(log.getDsName());
+        } finally {
+            writeUnlock();
         }
-        dsMgrProperty.write(out);
     }
 
     /**
-     * read from image.
-     *
-     * @param in
-     * @return
-     * @throws IOException
+     * Reply for alter ds name event.
      */
-    public static DataSourceMgr read(DataInput in) throws IOException {
-        if (Config.disable_cluster_feature) {
-            return null;
+    public void replyAlterDsName(DataSourceLog log) throws UserException {
+        writeLock();
+        try {
+            if (!nameToDataSource.containsKey(log.getDsName())) {
+                throw new DdlException("No datasource found with name: " + log.getDsName());
+            }
+            DataSourceIf ds = nameToDataSource.remove(log.getDsName());
+            ds.modifyDatasourceName(log.getNewDsName());
+            nameToDataSource.put(ds.getName(), ds);
+        } finally {
+            writeUnlock();
         }
-        DataSourceMgr mgr = new DataSourceMgr();
-        mgr.readFields(in);
-        return mgr;
     }
 
-    private void readFields(DataInput in) throws IOException {
-        int size = in.readInt();
-        for (int i = 0; i < size; ++i) {
-            ExternalDataSource extDs = ExternalDataSource.read(in);
-            idToDataSource.put(extDs.getId(), extDs);
-            nameToDataSource.put(extDs.getName(), extDs);
+    /**
+     * Reply for alter ds props event.
+     */
+    public void replyAlterDsProps(DataSourceLog log) throws UserException {
+        writeLock();
+        try {
+            if (!nameToDataSource.containsKey(log.getDsName())) {
+                throw new DdlException("No datasource found with name: " + log.getDsName());
+            }
+            if (!nameToDataSource.get(log.getDsName()).getType().equalsIgnoreCase(log.getNewProps().get("type"))) {
+                throw new DdlException("Can't modify the type of datasource property with name: " + log.getDsName());

Review Comment:
   I think we can analyze this error in `analyze()` of AlterDsStmt?



##########
fe/fe-core/src/main/java/org/apache/doris/datasource/DataSourceMgr.java:
##########
@@ -52,103 +66,180 @@ public DataSourceMgr() {
 
     private void initInternalDataSource() {
         internalDataSource = new InternalDataSource();
-        idToDataSource.put(internalDataSource.getId(), internalDataSource);
         nameToDataSource.put(internalDataSource.getName(), internalDataSource);
     }
 
-    private void registerNewDataSource(ExternalDataSource ds) {
-        // TODO
-    }
-
     public InternalDataSource getInternalDataSource() {
         return internalDataSource;
     }
 
+    private void writeLock() {
+        lock.writeLock().lock();
+    }
+
+    private void writeUnlock() {
+        lock.writeLock().unlock();
+    }
+
+    private void readLock() {
+        lock.readLock().lock();
+    }
+
+    private void readUnlock() {
+        lock.readLock().unlock();
+    }
+
     /**
-     * get data source by id.
-     *
-     * @param id
-     * @param e
-     * @param <E>
-     * @return
-     * @throws E
+     * Create and hold the datasource instance and write the meta log.
      */
-    public <E extends MetaNotFoundException> DataSourceIf getDataSourceOrException(long id,
-            java.util.function.Function<Long, E> e) throws E {
-        DataSourceIf ds = idToDataSource.get(id);
-        if (ds == null) {
-            throw e.apply(id);
+    public void createDatasource(CreateCatalogStmt stmt) throws UserException {
+        if (stmt.isSetIfNotExists() && nameToDataSource.containsKey(stmt.getCatalogName())) {
+            LOG.warn("Datasource {} is already exist.", stmt.getCatalogName());
+            return;
         }
-        return ds;
+        DataSourceLog log = DataSourceFactory.constructorDatasourceLog(stmt);
+        replyCreateDs(log);
+        Catalog.getCurrentCatalog().getEditLog().logDatasourceLog(OperationType.OP_CREATE_DS, log);
     }
 
     /**
-     * get data source by name.
-     *
-     * @param name
-     * @param e
-     * @param <E>
-     * @return
-     * @throws E
+     * Remove the datasource instance by name and write the meta log.
      */
-    public <E extends MetaNotFoundException> DataSourceIf getDataSourceOrException(String name,
-            java.util.function.Function<String, E> e) throws E {
-        DataSourceIf ds = nameToDataSource.get(name);
-        if (ds == null) {
-            throw e.apply(name);
+    public void dropDatasource(DropCatalogStmt stmt) throws UserException {
+        if (stmt.isSetIfExists() && !nameToDataSource.containsKey(stmt.getCatalogName())) {
+            LOG.warn("Non datasource {} is found.", stmt.getCatalogName());
+            return;
         }
-        return ds;
+        DataSourceLog log = DataSourceFactory.constructorDatasourceLog(stmt);
+        replyDropDs(log);

Review Comment:
   And all other `reply` words



##########
fe/fe-core/src/main/java/org/apache/doris/datasource/DataSourceMgr.java:
##########
@@ -52,103 +66,180 @@ public DataSourceMgr() {
 
     private void initInternalDataSource() {
         internalDataSource = new InternalDataSource();
-        idToDataSource.put(internalDataSource.getId(), internalDataSource);
         nameToDataSource.put(internalDataSource.getName(), internalDataSource);
     }
 
-    private void registerNewDataSource(ExternalDataSource ds) {
-        // TODO
-    }
-
     public InternalDataSource getInternalDataSource() {
         return internalDataSource;
     }
 
+    private void writeLock() {
+        lock.writeLock().lock();
+    }
+
+    private void writeUnlock() {
+        lock.writeLock().unlock();
+    }
+
+    private void readLock() {
+        lock.readLock().lock();
+    }
+
+    private void readUnlock() {
+        lock.readLock().unlock();
+    }
+
     /**
-     * get data source by id.
-     *
-     * @param id
-     * @param e
-     * @param <E>
-     * @return
-     * @throws E
+     * Create and hold the datasource instance and write the meta log.
      */
-    public <E extends MetaNotFoundException> DataSourceIf getDataSourceOrException(long id,
-            java.util.function.Function<Long, E> e) throws E {
-        DataSourceIf ds = idToDataSource.get(id);
-        if (ds == null) {
-            throw e.apply(id);
+    public void createDatasource(CreateCatalogStmt stmt) throws UserException {

Review Comment:
   For `create/drop/alter` ds, your main logic is like:
   ```
   DataSourceLog log = DataSourceFactory.xxx();
   replayXXX(log);
   Catalog.getCurrentCatalog().getEditLog().logDatasourceLog();
   ```
   
   It works but may have some side effect.
   Because in the `replayXXX` methods, there are some validation logic such as "check if ds exist", "check if property is valid".
   And these `replayXXX` methods are also be called in replay editlog logic.
   But in replay logic, there shouldn't be any exception thrown. So it is not suitable to put validation logic
   in `replayXXX` methods.
   `replayXXX` should only do the final metadata modification, other logic can be placed outside it.
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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