korlov42 commented on code in PR #4452:
URL: https://github.com/apache/ignite-3/pull/4452#discussion_r1776644840


##########
modules/cli/src/main/java/org/apache/ignite/internal/cli/Main.java:
##########
@@ -60,7 +60,7 @@ public static void main(String[] args) {
             AnsiConsole.systemInstall();
             initReplExecutor(micronautFactory);
             initQuestionAsker(micronautFactory);
-            if (args.length != 0 || !isatty()) { // do not enter REPL if input 
or output is redirected
+            if (args.length != 0 /*|| !isatty()*/) { // do not enter REPL if 
input or output is redirected

Review Comment:
   why do you commented out this? 



##########
modules/catalog-dsl/src/main/java/org/apache/ignite/internal/catalog/sql/SelectFromView.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.ignite.internal.catalog.sql;
+
+import static 
org.apache.ignite.internal.util.CompletableFutures.nullCompletedFuture;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Function;
+import org.apache.ignite.sql.IgniteSql;
+import org.apache.ignite.sql.SqlRow;
+import org.apache.ignite.sql.async.AsyncResultSet;
+
+class SelectFromView<T> extends AbstractCatalogQuery<List<T>> {
+    private final String viewName;
+
+    private final List<Option> whereOptions = new ArrayList<>();
+
+    private final Function<SqlRow, T> mapper;
+
+    SelectFromView(IgniteSql sql, String viewName, Option whereOption, 
Function<SqlRow, T> mapper) {
+        this(sql, viewName, List.of(whereOption), mapper);
+    }
+
+    SelectFromView(IgniteSql sql, String viewName, List<Option> whereOptions, 
Function<SqlRow, T> mapper) {
+        super(sql);
+        this.viewName = viewName;
+        this.whereOptions.addAll(whereOptions);
+        this.mapper = mapper;
+    }
+
+    @Override
+    public CompletableFuture<List<T>> executeAsync() {
+        return sql.executeAsync(null, toString()).thenCompose(resultSet -> {
+            List<T> result = new ArrayList<>();
+            return iterate(resultSet, result).thenApply(unused -> result);
+        });
+    }
+
+    private CompletableFuture<Void> iterate(AsyncResultSet<SqlRow> resultSet, 
List<T> result) {
+        for (SqlRow row : resultSet.currentPage()) {
+            result.add(mapper.apply(row));
+        }
+        if (resultSet.hasMorePages()) {
+            return resultSet.fetchNextPage().thenCompose(nextPage -> 
iterate(nextPage, result));
+        } else {
+            return nullCompletedFuture();
+        }
+    }
+
+    @Override
+    // Noop
+    protected List<T> result() {
+        return Collections.emptyList();
+    }
+
+    @Override
+    protected void accept(QueryContext ctx) {
+        ctx.sql("SELECT * FROM SYSTEM." + viewName + " ");
+
+        if (!whereOptions.isEmpty()) {
+            ctx.sql("WHERE ");
+            for (Option option : whereOptions) {
+                option.accept(ctx);
+            }
+        }
+
+        System.out.println("SELECT FROM VIEW");
+        System.out.println(ctx.getSql());

Review Comment:
   is it debug output?



##########
modules/catalog-dsl/src/main/java/org/apache/ignite/internal/catalog/sql/CreateFromDefinitionImpl.java:
##########
@@ -70,20 +71,21 @@ CreateFromDefinitionImpl from(ZoneDefinition def) {
             createZone.distributionAlgorithm(def.distributionAlgorithm());
         }
 
-        if (isGreaterThanZero(def.dataNodesAutoAdjust())) {
+        if (isPositive(def.dataNodesAutoAdjust())) {
             createZone.dataNodesAutoAdjust(def.dataNodesAutoAdjust());
         }
-        if (isGreaterThanZero(def.dataNodesAutoAdjustScaleUp())) {
+        if (isPositive(def.dataNodesAutoAdjustScaleUp())) {
             
createZone.dataNodesAutoAdjustScaleUp(def.dataNodesAutoAdjustScaleUp());
         }
-        if (isGreaterThanZero(def.dataNodesAutoAdjustScaleDown())) {
+        if (isPositive(def.dataNodesAutoAdjustScaleDown())) {
             
createZone.dataNodesAutoAdjustScaleDown(def.dataNodesAutoAdjustScaleDown());
         }
 
         if (!StringUtils.nullOrBlank(def.filter())) {
             createZone.filter(def.filter());
         }
-
+        System.out.println("Create");
+        System.out.println(this);

Review Comment:
   another debug output. Please clean it everywhere within the patch



##########
modules/catalog-dsl/build.gradle:
##########
@@ -26,6 +26,7 @@ dependencies {
     implementation libs.jetbrains.annotations
     implementation project(':ignite-api')
     implementation project(':ignite-core')
+    implementation project(':ignite-catalog')

Review Comment:
   why do you need this dependency?



##########
modules/catalog-dsl/src/main/java/org/apache/ignite/internal/catalog/sql/QueryUtils.java:
##########
@@ -53,6 +53,10 @@ static boolean isGreaterThanZero(Integer n) {
         return n != null && n > 0;
     }
 
+    static boolean isPositive(Integer n) {
+        return n != null && n >= 0;

Review Comment:
   actually, positive numbers are those which are strictly greater than zero. 
You may rename this method to `nonNegative` or, even better, 
`isGreaterThanOrEqualToZero` to make naming consistent with already existing 
method. Beside, it would be nice to provide descriptive javadoc for the method



##########
modules/catalog/src/main/java/org/apache/ignite/internal/catalog/systemviews/TablesSystemViewProvider.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.ignite.internal.catalog.systemviews;
+
+import static org.apache.ignite.internal.type.NativeTypes.BOOLEAN;
+import static org.apache.ignite.internal.type.NativeTypes.INT32;
+import static org.apache.ignite.internal.type.NativeTypes.STRING;
+import static org.apache.ignite.internal.type.NativeTypes.stringOf;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.Flow.Publisher;
+import java.util.function.Supplier;
+import org.apache.ignite.internal.catalog.Catalog;
+import org.apache.ignite.internal.catalog.CatalogSystemViewProvider;
+import 
org.apache.ignite.internal.catalog.descriptors.CatalogTableColumnDescriptor;
+import org.apache.ignite.internal.systemview.api.SystemView;
+import org.apache.ignite.internal.systemview.api.SystemViews;
+import org.apache.ignite.internal.util.SubscriptionUtils;
+
+/**
+ * Exposes information on tables.
+ *
+ * <ul>
+ *     <li>TABLES - available tables</li>
+ *     <li>TABLES_COLUMNS - columns of available tables.</li>
+ *     <li>TABLES_COLOCATION_COLUMNS - colocated columns of available 
tables.</li>
+ * </ul>
+ */
+public class TablesSystemViewProvider implements CatalogSystemViewProvider {
+    private static final int SYSTEM_VIEW_STRING_COLUMN_LENGTH = 
Short.MAX_VALUE;
+
+    @Override
+    public List<SystemView<?>> getView(Supplier<Catalog> catalogSupplier) {
+        return List.of(
+                getSystemViewView(catalogSupplier),
+                getSystemViewColocationColumnsView(catalogSupplier),
+                getSystemViewColumnsView(catalogSupplier)

Review Comment:
   it's better to rename it to get**Tables**View



##########
modules/catalog/src/main/java/org/apache/ignite/internal/catalog/systemviews/TablesSystemViewProvider.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.ignite.internal.catalog.systemviews;
+
+import static org.apache.ignite.internal.type.NativeTypes.BOOLEAN;
+import static org.apache.ignite.internal.type.NativeTypes.INT32;
+import static org.apache.ignite.internal.type.NativeTypes.STRING;
+import static org.apache.ignite.internal.type.NativeTypes.stringOf;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.Flow.Publisher;
+import java.util.function.Supplier;
+import org.apache.ignite.internal.catalog.Catalog;
+import org.apache.ignite.internal.catalog.CatalogSystemViewProvider;
+import 
org.apache.ignite.internal.catalog.descriptors.CatalogTableColumnDescriptor;
+import org.apache.ignite.internal.systemview.api.SystemView;
+import org.apache.ignite.internal.systemview.api.SystemViews;
+import org.apache.ignite.internal.util.SubscriptionUtils;
+
+/**
+ * Exposes information on tables.
+ *
+ * <ul>
+ *     <li>TABLES - available tables</li>
+ *     <li>TABLES_COLUMNS - columns of available tables.</li>
+ *     <li>TABLES_COLOCATION_COLUMNS - colocated columns of available 
tables.</li>
+ * </ul>
+ */
+public class TablesSystemViewProvider implements CatalogSystemViewProvider {
+    private static final int SYSTEM_VIEW_STRING_COLUMN_LENGTH = 
Short.MAX_VALUE;
+
+    @Override
+    public List<SystemView<?>> getView(Supplier<Catalog> catalogSupplier) {
+        return List.of(
+                getSystemViewView(catalogSupplier),
+                getSystemViewColocationColumnsView(catalogSupplier),
+                getSystemViewColumnsView(catalogSupplier)
+        );
+    }
+
+    private static SystemView<?> getSystemViewView(Supplier<Catalog> 
catalogSupplier) {
+        Iterable<Info> tablesData = () -> {
+            Catalog catalog = catalogSupplier.get();
+
+            return catalog.tables().stream().map(table -> {
+                String tableName = table.name();
+                String schemaName = 
Objects.requireNonNull(catalog.schema(table.schemaId()), "Schema must be not 
null.").name();
+                String zoneName = 
Objects.requireNonNull(catalog.zone(table.zoneId()), "Zone must be not 
null.").name();
+
+                int indexId = table.primaryKeyIndexId();
+
+                return new Info(tableName, schemaName, indexId, zoneName);
+            }).iterator();
+        };
+
+        Publisher<Info> viewDataPublisher = 
SubscriptionUtils.fromIterable(tablesData);
+
+        return SystemViews.<Info>clusterViewBuilder()
+                .name("TABLES")
+                .addColumn("SCHEMA", STRING, entry -> entry.schema)
+                .addColumn("NAME", STRING, entry -> entry.name)
+                .addColumn("PK_INDEX_ID", INT32, entry -> entry.pkIndexId)
+                .addColumn("ZONE", STRING, entry -> entry.zone)
+                .dataProvider(viewDataPublisher)
+                .build();
+    }
+
+    private static SystemView<?> getSystemViewColumnsView(Supplier<Catalog> 
catalogSupplier) {
+        Iterable<ColumnWithTableId> viewData = () -> {
+            Catalog catalog = catalogSupplier.get();
+
+            return catalog.tables().stream()
+                    .flatMap(table -> table.columns().stream()
+                            .map(columnDescriptor -> new ColumnWithTableId(
+                                    catalog.schema(table.schemaId()).name(),
+                                    table.name(),
+                                    columnDescriptor
+                                    )
+                            )
+                    )
+                    .iterator();
+        };
+
+        Publisher<ColumnWithTableId> viewDataPublisher = 
SubscriptionUtils.fromIterable(viewData);
+
+        return SystemViews.<ColumnWithTableId>clusterViewBuilder()
+                .name("TABLES_COLUMNS")
+                .addColumn("SCHEMA", STRING, entry -> entry.schema)
+                .addColumn("TABLE_NAME", STRING, entry -> entry.tableName)
+                .addColumn("COLUMN_NAME", STRING, entry -> 
entry.descriptor.name())
+                .addColumn("TYPE", STRING, entry -> 
entry.descriptor.type().name())
+                .addColumn("NULLABLE", BOOLEAN, entry -> 
entry.descriptor.nullable())
+                .addColumn("PRECISION", INT32, entry -> 
entry.descriptor.precision())
+                .addColumn("SCALE", INT32, entry -> entry.descriptor.scale())
+                .addColumn("LENGTH", INT32, entry -> entry.descriptor.length())
+                .dataProvider(viewDataPublisher)
+                .build();
+    }
+
+    private static SystemView<?> 
getSystemViewColocationColumnsView(Supplier<Catalog> catalogSupplier) {
+        Iterable<ColocationColumnsWithTable> viewData = () -> {
+            Catalog catalog = catalogSupplier.get();
+
+            return catalog.tables().stream()
+                    .flatMap(table -> table.colocationColumns().stream()
+                            .map(colocationColumn -> new 
ColocationColumnsWithTable(table.name(), colocationColumn))
+                    )
+                    .iterator();
+        };
+
+        Publisher<ColocationColumnsWithTable> viewDataPublisher = 
SubscriptionUtils.fromIterable(viewData);
+
+        return SystemViews.<ColocationColumnsWithTable>clusterViewBuilder()

Review Comment:
   order of columns in colocation key does matter, therefore we need to expose 
it in system view. Besides, table cannot be uniquely identified by name only, 
so name of the schema must be added as well.
   
    But why do we have it as separate view anyway? I would add more column to 
`TABLE_COLUMNS` with name `COLOCATION_KEY_INDEX` or similar in which I would 
add idea of column in colocation key or null if column doesn't belong to 
colocation key



##########
modules/api/src/main/java/org/apache/ignite/catalog/ColumnType.java:
##########
@@ -352,4 +352,6 @@ private ColumnType<T> length_(Integer length) {
         this.length = length;
         return this;
     }
+
+

Review Comment:
   unrelated changes



##########
modules/api/src/main/java/org/apache/ignite/catalog/IgniteCatalog.java:
##########
@@ -205,6 +221,10 @@ public interface IgniteCatalog {
      */
     void createZone(ZoneDefinition definition);
 
+    CompletableFuture<ZoneDefinition> zoneDefinitionAsync(String zoneName);
+
+    ZoneDefinition zoneDefinition(String zoneName);

Review Comment:
   please provide javadoc



##########
modules/catalog/src/main/java/org/apache/ignite/internal/catalog/systemviews/TablesSystemViewProvider.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.ignite.internal.catalog.systemviews;
+
+import static org.apache.ignite.internal.type.NativeTypes.BOOLEAN;
+import static org.apache.ignite.internal.type.NativeTypes.INT32;
+import static org.apache.ignite.internal.type.NativeTypes.STRING;
+import static org.apache.ignite.internal.type.NativeTypes.stringOf;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.Flow.Publisher;
+import java.util.function.Supplier;
+import org.apache.ignite.internal.catalog.Catalog;
+import org.apache.ignite.internal.catalog.CatalogSystemViewProvider;
+import 
org.apache.ignite.internal.catalog.descriptors.CatalogTableColumnDescriptor;
+import org.apache.ignite.internal.systemview.api.SystemView;
+import org.apache.ignite.internal.systemview.api.SystemViews;
+import org.apache.ignite.internal.util.SubscriptionUtils;
+
+/**
+ * Exposes information on tables.
+ *
+ * <ul>
+ *     <li>TABLES - available tables</li>
+ *     <li>TABLES_COLUMNS - columns of available tables.</li>
+ *     <li>TABLES_COLOCATION_COLUMNS - colocated columns of available 
tables.</li>
+ * </ul>
+ */
+public class TablesSystemViewProvider implements CatalogSystemViewProvider {
+    private static final int SYSTEM_VIEW_STRING_COLUMN_LENGTH = 
Short.MAX_VALUE;
+
+    @Override
+    public List<SystemView<?>> getView(Supplier<Catalog> catalogSupplier) {
+        return List.of(
+                getSystemViewView(catalogSupplier),
+                getSystemViewColocationColumnsView(catalogSupplier),
+                getSystemViewColumnsView(catalogSupplier)
+        );
+    }
+
+    private static SystemView<?> getSystemViewView(Supplier<Catalog> 
catalogSupplier) {
+        Iterable<Info> tablesData = () -> {
+            Catalog catalog = catalogSupplier.get();
+
+            return catalog.tables().stream().map(table -> {
+                String tableName = table.name();
+                String schemaName = 
Objects.requireNonNull(catalog.schema(table.schemaId()), "Schema must be not 
null.").name();
+                String zoneName = 
Objects.requireNonNull(catalog.zone(table.zoneId()), "Zone must be not 
null.").name();
+
+                int indexId = table.primaryKeyIndexId();
+
+                return new Info(tableName, schemaName, indexId, zoneName);
+            }).iterator();
+        };
+
+        Publisher<Info> viewDataPublisher = 
SubscriptionUtils.fromIterable(tablesData);
+
+        return SystemViews.<Info>clusterViewBuilder()
+                .name("TABLES")

Review Comment:
   let's add table id to every view



##########
modules/catalog/src/main/java/org/apache/ignite/internal/catalog/systemviews/TablesSystemViewProvider.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.ignite.internal.catalog.systemviews;
+
+import static org.apache.ignite.internal.type.NativeTypes.BOOLEAN;
+import static org.apache.ignite.internal.type.NativeTypes.INT32;
+import static org.apache.ignite.internal.type.NativeTypes.STRING;
+import static org.apache.ignite.internal.type.NativeTypes.stringOf;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.Flow.Publisher;
+import java.util.function.Supplier;
+import org.apache.ignite.internal.catalog.Catalog;
+import org.apache.ignite.internal.catalog.CatalogSystemViewProvider;
+import 
org.apache.ignite.internal.catalog.descriptors.CatalogTableColumnDescriptor;
+import org.apache.ignite.internal.systemview.api.SystemView;
+import org.apache.ignite.internal.systemview.api.SystemViews;
+import org.apache.ignite.internal.util.SubscriptionUtils;
+
+/**
+ * Exposes information on tables.
+ *
+ * <ul>
+ *     <li>TABLES - available tables</li>
+ *     <li>TABLES_COLUMNS - columns of available tables.</li>
+ *     <li>TABLES_COLOCATION_COLUMNS - colocated columns of available 
tables.</li>
+ * </ul>
+ */
+public class TablesSystemViewProvider implements CatalogSystemViewProvider {
+    private static final int SYSTEM_VIEW_STRING_COLUMN_LENGTH = 
Short.MAX_VALUE;
+
+    @Override
+    public List<SystemView<?>> getView(Supplier<Catalog> catalogSupplier) {
+        return List.of(
+                getSystemViewView(catalogSupplier),
+                getSystemViewColocationColumnsView(catalogSupplier),
+                getSystemViewColumnsView(catalogSupplier)
+        );
+    }
+
+    private static SystemView<?> getSystemViewView(Supplier<Catalog> 
catalogSupplier) {
+        Iterable<Info> tablesData = () -> {
+            Catalog catalog = catalogSupplier.get();
+
+            return catalog.tables().stream().map(table -> {
+                String tableName = table.name();
+                String schemaName = 
Objects.requireNonNull(catalog.schema(table.schemaId()), "Schema must be not 
null.").name();
+                String zoneName = 
Objects.requireNonNull(catalog.zone(table.zoneId()), "Zone must be not 
null.").name();
+
+                int indexId = table.primaryKeyIndexId();
+
+                return new Info(tableName, schemaName, indexId, zoneName);
+            }).iterator();
+        };
+
+        Publisher<Info> viewDataPublisher = 
SubscriptionUtils.fromIterable(tablesData);
+
+        return SystemViews.<Info>clusterViewBuilder()
+                .name("TABLES")
+                .addColumn("SCHEMA", STRING, entry -> entry.schema)
+                .addColumn("NAME", STRING, entry -> entry.name)
+                .addColumn("PK_INDEX_ID", INT32, entry -> entry.pkIndexId)
+                .addColumn("ZONE", STRING, entry -> entry.zone)
+                .dataProvider(viewDataPublisher)
+                .build();
+    }
+
+    private static SystemView<?> getSystemViewColumnsView(Supplier<Catalog> 
catalogSupplier) {
+        Iterable<ColumnWithTableId> viewData = () -> {
+            Catalog catalog = catalogSupplier.get();
+
+            return catalog.tables().stream()
+                    .flatMap(table -> table.columns().stream()
+                            .map(columnDescriptor -> new ColumnWithTableId(
+                                    catalog.schema(table.schemaId()).name(),
+                                    table.name(),
+                                    columnDescriptor
+                                    )
+                            )
+                    )
+                    .iterator();
+        };
+
+        Publisher<ColumnWithTableId> viewDataPublisher = 
SubscriptionUtils.fromIterable(viewData);
+
+        return SystemViews.<ColumnWithTableId>clusterViewBuilder()
+                .name("TABLES_COLUMNS")

Review Comment:
   ```suggestion
                   .name("TABLE_COLUMNS")
   ```



##########
modules/catalog/src/main/java/org/apache/ignite/internal/catalog/systemviews/TablesSystemViewProvider.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.ignite.internal.catalog.systemviews;
+
+import static org.apache.ignite.internal.type.NativeTypes.BOOLEAN;
+import static org.apache.ignite.internal.type.NativeTypes.INT32;
+import static org.apache.ignite.internal.type.NativeTypes.STRING;
+import static org.apache.ignite.internal.type.NativeTypes.stringOf;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.Flow.Publisher;
+import java.util.function.Supplier;
+import org.apache.ignite.internal.catalog.Catalog;
+import org.apache.ignite.internal.catalog.CatalogSystemViewProvider;
+import 
org.apache.ignite.internal.catalog.descriptors.CatalogTableColumnDescriptor;
+import org.apache.ignite.internal.systemview.api.SystemView;
+import org.apache.ignite.internal.systemview.api.SystemViews;
+import org.apache.ignite.internal.util.SubscriptionUtils;
+
+/**
+ * Exposes information on tables.
+ *
+ * <ul>
+ *     <li>TABLES - available tables</li>
+ *     <li>TABLES_COLUMNS - columns of available tables.</li>
+ *     <li>TABLES_COLOCATION_COLUMNS - colocated columns of available 
tables.</li>
+ * </ul>
+ */
+public class TablesSystemViewProvider implements CatalogSystemViewProvider {
+    private static final int SYSTEM_VIEW_STRING_COLUMN_LENGTH = 
Short.MAX_VALUE;
+
+    @Override
+    public List<SystemView<?>> getView(Supplier<Catalog> catalogSupplier) {
+        return List.of(
+                getSystemViewView(catalogSupplier),
+                getSystemViewColocationColumnsView(catalogSupplier),
+                getSystemViewColumnsView(catalogSupplier)
+        );
+    }
+
+    private static SystemView<?> getSystemViewView(Supplier<Catalog> 
catalogSupplier) {
+        Iterable<Info> tablesData = () -> {
+            Catalog catalog = catalogSupplier.get();
+
+            return catalog.tables().stream().map(table -> {
+                String tableName = table.name();
+                String schemaName = 
Objects.requireNonNull(catalog.schema(table.schemaId()), "Schema must be not 
null.").name();
+                String zoneName = 
Objects.requireNonNull(catalog.zone(table.zoneId()), "Zone must be not 
null.").name();
+
+                int indexId = table.primaryKeyIndexId();
+
+                return new Info(tableName, schemaName, indexId, zoneName);
+            }).iterator();
+        };
+
+        Publisher<Info> viewDataPublisher = 
SubscriptionUtils.fromIterable(tablesData);
+
+        return SystemViews.<Info>clusterViewBuilder()
+                .name("TABLES")
+                .addColumn("SCHEMA", STRING, entry -> entry.schema)
+                .addColumn("NAME", STRING, entry -> entry.name)
+                .addColumn("PK_INDEX_ID", INT32, entry -> entry.pkIndexId)
+                .addColumn("ZONE", STRING, entry -> entry.zone)
+                .dataProvider(viewDataPublisher)
+                .build();
+    }
+
+    private static SystemView<?> getSystemViewColumnsView(Supplier<Catalog> 
catalogSupplier) {
+        Iterable<ColumnWithTableId> viewData = () -> {
+            Catalog catalog = catalogSupplier.get();
+
+            return catalog.tables().stream()
+                    .flatMap(table -> table.columns().stream()
+                            .map(columnDescriptor -> new ColumnWithTableId(
+                                    catalog.schema(table.schemaId()).name(),
+                                    table.name(),
+                                    columnDescriptor
+                                    )
+                            )
+                    )
+                    .iterator();
+        };
+
+        Publisher<ColumnWithTableId> viewDataPublisher = 
SubscriptionUtils.fromIterable(viewData);
+
+        return SystemViews.<ColumnWithTableId>clusterViewBuilder()
+                .name("TABLES_COLUMNS")
+                .addColumn("SCHEMA", STRING, entry -> entry.schema)
+                .addColumn("TABLE_NAME", STRING, entry -> entry.tableName)
+                .addColumn("COLUMN_NAME", STRING, entry -> 
entry.descriptor.name())
+                .addColumn("TYPE", STRING, entry -> 
entry.descriptor.type().name())
+                .addColumn("NULLABLE", BOOLEAN, entry -> 
entry.descriptor.nullable())
+                .addColumn("PRECISION", INT32, entry -> 
entry.descriptor.precision())
+                .addColumn("SCALE", INT32, entry -> entry.descriptor.scale())
+                .addColumn("LENGTH", INT32, entry -> entry.descriptor.length())
+                .dataProvider(viewDataPublisher)
+                .build();
+    }
+
+    private static SystemView<?> 
getSystemViewColocationColumnsView(Supplier<Catalog> catalogSupplier) {
+        Iterable<ColocationColumnsWithTable> viewData = () -> {
+            Catalog catalog = catalogSupplier.get();
+
+            return catalog.tables().stream()
+                    .flatMap(table -> table.colocationColumns().stream()
+                            .map(colocationColumn -> new 
ColocationColumnsWithTable(table.name(), colocationColumn))
+                    )
+                    .iterator();
+        };
+
+        Publisher<ColocationColumnsWithTable> viewDataPublisher = 
SubscriptionUtils.fromIterable(viewData);
+
+        return SystemViews.<ColocationColumnsWithTable>clusterViewBuilder()
+                .name("TABLES_COLOCATION_COLUMNS")

Review Comment:
   ```suggestion
                   .name("TABLE_COLOCATION_COLUMNS")
   ```



##########
modules/catalog/src/main/java/org/apache/ignite/internal/catalog/systemviews/TablesSystemViewProvider.java:
##########
@@ -0,0 +1,172 @@
+/*
+ * 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.ignite.internal.catalog.systemviews;
+
+import static org.apache.ignite.internal.type.NativeTypes.BOOLEAN;
+import static org.apache.ignite.internal.type.NativeTypes.INT32;
+import static org.apache.ignite.internal.type.NativeTypes.STRING;
+import static org.apache.ignite.internal.type.NativeTypes.stringOf;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.Flow.Publisher;
+import java.util.function.Supplier;
+import org.apache.ignite.internal.catalog.Catalog;
+import org.apache.ignite.internal.catalog.CatalogSystemViewProvider;
+import 
org.apache.ignite.internal.catalog.descriptors.CatalogTableColumnDescriptor;
+import org.apache.ignite.internal.systemview.api.SystemView;
+import org.apache.ignite.internal.systemview.api.SystemViews;
+import org.apache.ignite.internal.util.SubscriptionUtils;
+
+/**
+ * Exposes information on tables.
+ *
+ * <ul>
+ *     <li>TABLES - available tables</li>
+ *     <li>TABLES_COLUMNS - columns of available tables.</li>
+ *     <li>TABLES_COLOCATION_COLUMNS - colocated columns of available 
tables.</li>
+ * </ul>
+ */
+public class TablesSystemViewProvider implements CatalogSystemViewProvider {
+    private static final int SYSTEM_VIEW_STRING_COLUMN_LENGTH = 
Short.MAX_VALUE;
+
+    @Override
+    public List<SystemView<?>> getView(Supplier<Catalog> catalogSupplier) {
+        return List.of(
+                getSystemViewView(catalogSupplier),
+                getSystemViewColocationColumnsView(catalogSupplier),
+                getSystemViewColumnsView(catalogSupplier)
+        );
+    }
+
+    private static SystemView<?> getSystemViewView(Supplier<Catalog> 
catalogSupplier) {
+        Iterable<Info> tablesData = () -> {
+            Catalog catalog = catalogSupplier.get();
+
+            return catalog.tables().stream().map(table -> {
+                String tableName = table.name();
+                String schemaName = 
Objects.requireNonNull(catalog.schema(table.schemaId()), "Schema must be not 
null.").name();
+                String zoneName = 
Objects.requireNonNull(catalog.zone(table.zoneId()), "Zone must be not 
null.").name();
+
+                int indexId = table.primaryKeyIndexId();
+
+                return new Info(tableName, schemaName, indexId, zoneName);
+            }).iterator();
+        };
+
+        Publisher<Info> viewDataPublisher = 
SubscriptionUtils.fromIterable(tablesData);
+
+        return SystemViews.<Info>clusterViewBuilder()
+                .name("TABLES")
+                .addColumn("SCHEMA", STRING, entry -> entry.schema)
+                .addColumn("NAME", STRING, entry -> entry.name)
+                .addColumn("PK_INDEX_ID", INT32, entry -> entry.pkIndexId)
+                .addColumn("ZONE", STRING, entry -> entry.zone)

Review Comment:
   let's also add storageProfile



##########
modules/catalog-dsl/src/main/java/org/apache/ignite/internal/catalog/sql/Option.java:
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.ignite.internal.catalog.sql;
+
+class Option extends QueryPart {
+    private final String name;
+
+    private final Object value;
+
+    private Option(String name, Object value) {
+        this.name = name;
+        this.value = value;
+    }
+
+    public static Option primaryZone(String zone) {
+        return new Option("PRIMARY_ZONE", zone.toUpperCase());
+    }
+
+    public static Option partitions(Integer partitions) {
+        return new Option("PARTITIONS", partitions);
+    }
+
+    public static Option replicas(Integer replicas) {
+        return new Option("REPLICAS", replicas);
+    }
+
+    public static Option dataNodesAutoAdjust(Integer adjust) {
+        return new Option("DATA_NODES_AUTO_ADJUST", adjust);
+    }
+
+    public static Option dataNodesAutoAdjustScaleUp(Integer adjust) {
+        return new Option("DATA_NODES_AUTO_ADJUST_SCALE_UP", adjust);
+    }
+
+    public static Option dataNodesAutoAdjustScaleDown(Integer adjust) {
+        return new Option("DATA_NODES_AUTO_ADJUST_SCALE_DOWN", adjust);
+    }
+
+    public static Option distributionAlgorithm(String distributionAlgorithm) {
+        return new Option("DISTRIBUTION_ALGORITHM", distributionAlgorithm);
+    }
+
+    public static Option dataRegion(String dataRegion) {
+        return new Option("DATAREGION", dataRegion);
+    }
+
+    public static Option storageProfiles(String storageProfiles) {
+        return new Option("STORAGE_PROFILES", storageProfiles);
+    }
+
+    public static Option name(String name) {
+        return new Option("NAME", name);
+    }
+
+    public static Option tableName(String tableName) {
+        return new Option("TABLE_NAME", tableName);
+    }
+
+    public static Option zoneName(String zoneName) {
+        return new Option("ZONE_NAME", zoneName);
+    }
+
+    public static Option indexId(int indexId) {
+        return new Option("INDEX_ID", indexId);
+    }
+
+    public static Option filter(String filter) {
+        return new Option("DATA_NODES_FILTER", filter);
+    }
+
+    @Override
+    protected void accept(QueryContext ctx) {
+        ctx.sql(name).sql("=");
+        boolean isNeedsQuotes = value instanceof String;
+        if (isNeedsQuotes) {
+            ctx.sql("'").sql(value.toString()).sql("'");

Review Comment:
   what if value itself contains a quote?



##########
modules/api/src/main/java/org/apache/ignite/catalog/definitions/ZoneDefinition.java:
##########
@@ -176,6 +176,22 @@ public Builder toBuilder() {
         return new Builder(this);
     }
 
+    @Override
+    public String toString() {

Review Comment:
   is it debug-only code? 



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to