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/15 02:31:56 UTC

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

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


##########
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:
   `analyze` can't get the type of this datasource. But I will move it out of replay Funciton



-- 
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