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/17 06:40:48 UTC

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

Jibing-Li commented on code in PR #10033:
URL: https://github.com/apache/incubator-doris/pull/10033#discussion_r899809492


##########
fe/fe-core/src/main/java/org/apache/doris/datasource/DataSourceMgr.java:
##########
@@ -52,103 +67,182 @@ public DataSourceMgr() {
 
     private void initInternalDataSource() {
         internalDataSource = new InternalDataSource();
-        idToDataSource.put(internalDataSource.getId(), internalDataSource);
-        nameToDataSource.put(internalDataSource.getName(), internalDataSource);
-    }
-
-    private void registerNewDataSource(ExternalDataSource ds) {
-        // TODO
+        nameToCatalogs.put(internalDataSource.getName(), internalDataSource);
     }
 
     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 catalog 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 createCatalog(CreateCatalogStmt stmt) throws UserException {
+        if (stmt.isSetIfNotExists() && nameToCatalogs.containsKey(stmt.getCatalogName())) {
+            LOG.warn("Catalog {} is already exist.", stmt.getCatalogName());
+            return;
         }
-        return ds;
+        if (nameToCatalogs.containsKey(stmt.getCatalogName())) {
+            throw new DdlException("Catalog had already exist with name: " + stmt.getCatalogName());
+        }
+        CatalogLog log = CatalogFactory.constructorCatalogLog(stmt);
+        replayCreateCatalog(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 catalog 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 dropCatalog(DropCatalogStmt stmt) throws UserException {
+        if (stmt.isSetIfExists() && !nameToCatalogs.containsKey(stmt.getCatalogName())) {
+            LOG.warn("Non catalog {} is found.", stmt.getCatalogName());
+            return;
+        }
+        if (!nameToCatalogs.containsKey(stmt.getCatalogName())) {
+            throw new DdlException("No catalog found with name: " + stmt.getCatalogName());
         }
-        return ds;
+        CatalogLog log = CatalogFactory.constructorCatalogLog(stmt);
+        replayDropCatalog(log);
+        Catalog.getCurrentCatalog().getEditLog().logDatasourceLog(OperationType.OP_DROP_DS, log);
     }
 
-    public boolean hasDataSource(String name) {
-        return nameToDataSource.containsKey(name);
+    /**
+     * Modify the catalog name into a new one and write the meta log.
+     */
+    public void alterCatalogName(AlterCatalogNameStmt stmt) throws UserException {
+        if (!nameToCatalogs.containsKey(stmt.getCatalogName())) {
+            throw new DdlException("No catalog found with name: " + stmt.getCatalogName());
+        }
+        CatalogLog log = CatalogFactory.constructorCatalogLog(stmt);
+        replayAlterCatalogName(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 catalog property and write the meta log.
+     */
+    public void alterCatalogProps(AlterCatalogPropertyStmt stmt) throws UserException {
+        if (!nameToCatalogs.containsKey(stmt.getCatalogName())) {
+            throw new DdlException("No catalog found with name: " + stmt.getCatalogName());
         }
-        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;
+        if (!nameToCatalogs.get(stmt.getCatalogName())
+                .getType().equalsIgnoreCase(stmt.getNewProperties().get("type"))) {
+            throw new DdlException("Can't modify the type of catalog property with name: " + stmt.getCatalogName());
         }
-        out.writeInt(size);
-        for (DataSourceIf ds : idToDataSource.values()) {
-            if (ds.getId() == InternalDataSource.INTERNAL_DS_ID) {
-                continue;
+        CatalogLog log = CatalogFactory.constructorCatalogLog(stmt);
+        replayAlterCatalogProps(log);
+        Catalog.getCurrentCatalog().getEditLog().logDatasourceLog(OperationType.OP_ALTER_DS_PROPS, log);
+    }
+
+    /**
+     * List all catalog or get the special catalog with a name.
+     */
+    public ShowResultSet showCatalogs(ShowCatalogStmt showStmt) throws AnalysisException {
+        List<List<String>> rows = Lists.newArrayList();
+        readLock();
+        try {
+            if (showStmt.getCatalogName() == null) {
+                for (DataSourceIf ds : nameToCatalogs.values()) {
+                    List<String> row = Lists.newArrayList();
+                    row.add(ds.getName());
+                    row.add(ds.getType());
+                    rows.add(row);
+                }
+            } else {
+                if (!nameToCatalogs.containsKey(showStmt.getCatalogName())) {
+                    throw new AnalysisException("No catalog found with name: " + showStmt.getCatalogName());
+                }
+                DataSourceIf ds = nameToCatalogs.get(showStmt.getCatalogName());
+                for (Map.Entry<String, String>  elem : ds.getProperties().entrySet()) {
+                    List<String> row = Lists.newArrayList();
+                    row.add(elem.getKey());
+                    row.add(elem.getValue());
+                    rows.add(row);
+                }
             }
-            ExternalDataSource extDs = (ExternalDataSource) ds;
-            extDs.write(out);
+        } finally {
+            readUnlock();
         }
-        dsMgrProperty.write(out);
+
+        return new ShowResultSet(showStmt.getMetaData(), rows);
     }
 
     /**
-     * read from image.
-     *
-     * @param in
-     * @return
-     * @throws IOException
+     * Reply for create catalog event.
      */
-    public static DataSourceMgr read(DataInput in) throws IOException {
-        if (Config.disable_cluster_feature) {
-            return null;
+    public void replayCreateCatalog(CatalogLog log) {
+        writeLock();
+        try {
+            DataSourceIf ds = CatalogFactory.constructorFromLog(log);
+            nameToCatalogs.put(ds.getName(), ds);
+        } finally {
+            writeUnlock();
+        }
+    }
+
+    /**
+     * Reply for drop catalog event.
+     */
+    public void replayDropCatalog(CatalogLog log) {
+        writeLock();
+        try {
+            nameToCatalogs.remove(log.getCatalogName());
+        } finally {
+            writeUnlock();
+        }
+    }
+
+    /**
+     * Reply for alter catalog name event.
+     */
+    public void replayAlterCatalogName(CatalogLog log) {
+        writeLock();
+        try {
+            DataSourceIf ds = nameToCatalogs.remove(log.getCatalogName());
+            ds.modifyDatasourceName(log.getNewCatalogName());
+            nameToCatalogs.put(ds.getName(), ds);

Review Comment:
   Why not simply get and set?
   ```
   DataSourceIf ds = nameToCatalogs.get(log.getCatalogName());
   ds.modifyDatasourceName(log.getNewCatalogName());
   ```



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