You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2021/05/18 17:31:05 UTC

[GitHub] [ignite-3] Berkof opened a new pull request #132: GG-14290: SchemaDescriptor and ConfigurationSchema converters.

Berkof opened a new pull request #132:
URL: https://github.com/apache/ignite-3/pull/132


   


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

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



[GitHub] [ignite-3] AMashenkov commented on a change in pull request #132: GG-14290: SchemaDescriptor and ConfigurationSchema converters.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #132:
URL: https://github.com/apache/ignite-3/pull/132#discussion_r635019100



##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnChange;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexChange;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.schemas.table.TablesChange;
+import org.apache.ignite.configuration.tree.NamedListView;
+import org.apache.ignite.internal.schema.ColumnImpl;
+import org.apache.ignite.internal.schema.HashIndexImpl;
+import org.apache.ignite.internal.schema.PartialIndexImpl;
+import org.apache.ignite.internal.schema.PrimaryIndexImpl;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.SortedIndexColumnImpl;
+import org.apache.ignite.internal.schema.SortedIndexImpl;
+import org.apache.ignite.schema.Column;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.IndexColumn;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
+import org.apache.ignite.schema.SortedIndexColumn;
+import org.apache.ignite.schema.TableIndex;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/** Configuration to schema and vice versa converter. */
+public class SchemaConfigurationConverter {
+    /** Hash index type. */
+    private static final String HASH_TYPE = "HASH";
+
+    /** Sorted index type. */
+    private static final String SORTED_TYPE = "SORTED";
+
+    /** Partial index type. */
+    private static final String PARTIAL_TYPE = "PARTIAL";
+
+    /** Primary key index type. */
+    private static final String PK_TYPE = "PK";
+
+    /** Types map. */
+    private static Map<String, ColumnType> types = new HashMap<>();
+
+    static {
+        putType(ColumnType.INT8);
+        putType(ColumnType.INT16);
+        putType(ColumnType.INT32);
+        putType(ColumnType.INT64);
+        putType(ColumnType.UINT8);
+        putType(ColumnType.UINT16);
+        putType(ColumnType.UINT32);
+        putType(ColumnType.UINT64);
+        putType(ColumnType.FLOAT);
+        putType(ColumnType.DOUBLE);
+        putType(ColumnType.UUID);
+        putType(ColumnType.string());
+        putType(ColumnType.blobOf());
+        // TODO; handle length for some types
+        //putType(ColumnType.bitmaskOf());
+    }
+
+    private static void putType(ColumnType type) {
+        types.put(type.typeSpec().name(), type);
+    }
+
+    /**
+     * Convert SortedIndexColumn to IndexColumnChange.
+     * @param col IndexColumnChange.
+     * @param colInit IndexColumnChange to fullfill.
+     */
+    public static void convert(SortedIndexColumn col, IndexColumnChange colInit) {
+        colInit.changeName(col.name());
+        colInit.changeAsc(col.asc());
+    }
+
+    /**
+     * Convert IndexColumnView to SortedIndexColumn.
+     *
+     * @param colCfg IndexColumnView.
+     * @return SortedIndexColumn.
+     */
+    public static SortedIndexColumn convert(IndexColumnView colCfg) {
+        return new SortedIndexColumnImpl(colCfg.name(), colCfg.asc());
+    }
+
+    /**
+     * Convert TableIndex to TableIndexChange.
+     *
+     * @param idx TableIndex.
+     * @param idxChg TableIndexChange to fullfill.
+     */
+    public static void convert(TableIndex idx, TableIndexChange idxChg) {
+        idxChg.changeName(idx.name());
+        idxChg.changeType(idx.type());
+
+        if (HASH_TYPE.equals(idx.type())) {
+            HashIndex hashIdx = (HashIndex)idx;
+
+            String[] colNames = hashIdx.columns().stream().map(IndexColumn::name).toArray(String[]::new);
+            idxChg.changeColNames(colNames);
+        }
+        else if (PARTIAL_TYPE.equals(idx.type())) {
+            PartialIndex partIdx = (PartialIndex)idx;
+
+            idxChg.changeUniq(partIdx.unique());
+            idxChg.changeExpr(partIdx.expr());
+
+            idxChg.changeColumns(colsChg -> {
+                for (SortedIndexColumn col : partIdx.columns())
+                    colsChg.create(col.name(), colInit -> convert(col, colInit));
+            });
+
+        }
+        else if (SORTED_TYPE.equals(idx.type())) {
+            SortedIndex sortIdx = (SortedIndex)idx;
+            idxChg.changeUniq(sortIdx.unique());
+
+            idxChg.changeColumns(colsInit -> {
+                for (SortedIndexColumn col : sortIdx.columns())
+                    colsInit.create(col.name(), colInit -> convert(col, colInit));
+            });
+        }
+        else if (PK_TYPE.equals(idx.type())) {
+            PrimaryIndex primIdx = (PrimaryIndex)idx;
+
+            idxChg.changeColumns(colsInit -> {
+                for (SortedIndexColumn col : primIdx.columns())
+                    colsInit.create(col.name(), colInit -> convert(col, colInit));
+            });
+
+            idxChg.changeAffinityColumns(primIdx.affinityColumns().toArray(new String[primIdx.affinityColumns().size()]));
+        }
+        else throw new IllegalArgumentException("Unknown index type " + idx.type());
+    }
+
+    /**
+     * Convert TableIndexView into TableIndex.
+     *
+     * @param idxView TableIndexView.
+     * @return TableIndex.
+     */
+    public static TableIndex convert(TableIndexView idxView) {
+        String name = idxView.name();
+        String type = idxView.type();
+
+        if (type.equals("HASH")) {
+            String[] cols = idxView.colNames();
+
+            return new HashIndexImpl(name, cols);
+        }
+        else if (type.equals("SORTED") || type.equals("PARTIAL") || type.equals("PK")) {
+            if (type.equals("PARTIAL")) {
+                boolean uniq = idxView.uniq();
+                String expr = idxView.expr();
+                NamedListView<? extends IndexColumnView> colsView = idxView.columns();
+                List<SortedIndexColumn> cols = new ArrayList<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(colsView.get(key));
+                    cols.add(col);
+                }
+
+                return new PartialIndexImpl(name, cols, uniq, expr);
+            }
+            if (type.equals("SORTED")) {
+                boolean uniq = idxView.uniq();
+                List<SortedIndexColumn> cols = new ArrayList<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    cols.add(col);
+                }
+
+                return new SortedIndexImpl(name, cols, uniq);
+            }
+            if (type.equals("PK")) {
+                List<SortedIndexColumn> cols = new ArrayList<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    cols.add(col);
+                }
+                String[] affCols = idxView.colNames();
+
+                return new PrimaryIndexImpl(cols, Arrays.stream(affCols).collect(Collectors.toList()));
+            }
+
+            return null;
+        }
+        else
+            throw new IllegalArgumentException("Unknown type " + type);
+    }
+
+    /**
+     * Convert ColumnType to ColumnTypeChange.
+     *
+     * @param colType ColumnType.
+     * @param colTypeChg ColumnTypeChange to fullfill.
+     */
+    public static void convert(ColumnType colType, ColumnTypeChange colTypeChg) {
+        colTypeChg.changeType(colType.typeSpec().name());
+        // TODO varlen types

Review comment:
       TODO with no ticket.

##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnChange;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexChange;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.schemas.table.TablesChange;
+import org.apache.ignite.configuration.tree.NamedListView;
+import org.apache.ignite.internal.schema.ColumnImpl;
+import org.apache.ignite.internal.schema.HashIndexImpl;
+import org.apache.ignite.internal.schema.PartialIndexImpl;
+import org.apache.ignite.internal.schema.PrimaryIndexImpl;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.SortedIndexColumnImpl;
+import org.apache.ignite.internal.schema.SortedIndexImpl;
+import org.apache.ignite.schema.Column;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.IndexColumn;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
+import org.apache.ignite.schema.SortedIndexColumn;
+import org.apache.ignite.schema.TableIndex;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/** Configuration to schema and vice versa converter. */
+public class SchemaConfigurationConverter {
+    /** Hash index type. */
+    private static final String HASH_TYPE = "HASH";
+
+    /** Sorted index type. */
+    private static final String SORTED_TYPE = "SORTED";
+
+    /** Partial index type. */
+    private static final String PARTIAL_TYPE = "PARTIAL";
+
+    /** Primary key index type. */
+    private static final String PK_TYPE = "PK";
+
+    /** Types map. */
+    private static Map<String, ColumnType> types = new HashMap<>();
+
+    static {
+        putType(ColumnType.INT8);
+        putType(ColumnType.INT16);
+        putType(ColumnType.INT32);
+        putType(ColumnType.INT64);
+        putType(ColumnType.UINT8);
+        putType(ColumnType.UINT16);
+        putType(ColumnType.UINT32);
+        putType(ColumnType.UINT64);
+        putType(ColumnType.FLOAT);
+        putType(ColumnType.DOUBLE);
+        putType(ColumnType.UUID);
+        putType(ColumnType.string());
+        putType(ColumnType.blobOf());
+        // TODO; handle length for some types
+        //putType(ColumnType.bitmaskOf());
+    }
+
+    private static void putType(ColumnType type) {
+        types.put(type.typeSpec().name(), type);
+    }
+
+    /**
+     * Convert SortedIndexColumn to IndexColumnChange.
+     * @param col IndexColumnChange.
+     * @param colInit IndexColumnChange to fullfill.
+     */
+    public static void convert(SortedIndexColumn col, IndexColumnChange colInit) {
+        colInit.changeName(col.name());
+        colInit.changeAsc(col.asc());
+    }
+
+    /**
+     * Convert IndexColumnView to SortedIndexColumn.
+     *
+     * @param colCfg IndexColumnView.
+     * @return SortedIndexColumn.
+     */
+    public static SortedIndexColumn convert(IndexColumnView colCfg) {
+        return new SortedIndexColumnImpl(colCfg.name(), colCfg.asc());
+    }
+
+    /**
+     * Convert TableIndex to TableIndexChange.
+     *
+     * @param idx TableIndex.
+     * @param idxChg TableIndexChange to fullfill.
+     */
+    public static void convert(TableIndex idx, TableIndexChange idxChg) {
+        idxChg.changeName(idx.name());
+        idxChg.changeType(idx.type());
+
+        if (HASH_TYPE.equals(idx.type())) {
+            HashIndex hashIdx = (HashIndex)idx;
+
+            String[] colNames = hashIdx.columns().stream().map(IndexColumn::name).toArray(String[]::new);
+            idxChg.changeColNames(colNames);
+        }
+        else if (PARTIAL_TYPE.equals(idx.type())) {
+            PartialIndex partIdx = (PartialIndex)idx;
+
+            idxChg.changeUniq(partIdx.unique());
+            idxChg.changeExpr(partIdx.expr());
+
+            idxChg.changeColumns(colsChg -> {
+                for (SortedIndexColumn col : partIdx.columns())
+                    colsChg.create(col.name(), colInit -> convert(col, colInit));
+            });
+
+        }
+        else if (SORTED_TYPE.equals(idx.type())) {
+            SortedIndex sortIdx = (SortedIndex)idx;
+            idxChg.changeUniq(sortIdx.unique());
+
+            idxChg.changeColumns(colsInit -> {
+                for (SortedIndexColumn col : sortIdx.columns())
+                    colsInit.create(col.name(), colInit -> convert(col, colInit));
+            });
+        }
+        else if (PK_TYPE.equals(idx.type())) {
+            PrimaryIndex primIdx = (PrimaryIndex)idx;
+
+            idxChg.changeColumns(colsInit -> {
+                for (SortedIndexColumn col : primIdx.columns())
+                    colsInit.create(col.name(), colInit -> convert(col, colInit));
+            });
+
+            idxChg.changeAffinityColumns(primIdx.affinityColumns().toArray(new String[primIdx.affinityColumns().size()]));
+        }
+        else throw new IllegalArgumentException("Unknown index type " + idx.type());
+    }
+
+    /**
+     * Convert TableIndexView into TableIndex.
+     *
+     * @param idxView TableIndexView.
+     * @return TableIndex.
+     */
+    public static TableIndex convert(TableIndexView idxView) {
+        String name = idxView.name();
+        String type = idxView.type();
+
+        if (type.equals("HASH")) {
+            String[] cols = idxView.colNames();
+
+            return new HashIndexImpl(name, cols);
+        }
+        else if (type.equals("SORTED") || type.equals("PARTIAL") || type.equals("PK")) {
+            if (type.equals("PARTIAL")) {
+                boolean uniq = idxView.uniq();
+                String expr = idxView.expr();
+                NamedListView<? extends IndexColumnView> colsView = idxView.columns();
+                List<SortedIndexColumn> cols = new ArrayList<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(colsView.get(key));
+                    cols.add(col);
+                }
+
+                return new PartialIndexImpl(name, cols, uniq, expr);
+            }
+            if (type.equals("SORTED")) {
+                boolean uniq = idxView.uniq();
+                List<SortedIndexColumn> cols = new ArrayList<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    cols.add(col);
+                }
+
+                return new SortedIndexImpl(name, cols, uniq);
+            }
+            if (type.equals("PK")) {
+                List<SortedIndexColumn> cols = new ArrayList<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    cols.add(col);
+                }
+                String[] affCols = idxView.colNames();
+
+                return new PrimaryIndexImpl(cols, Arrays.stream(affCols).collect(Collectors.toList()));
+            }
+
+            return null;
+        }
+        else
+            throw new IllegalArgumentException("Unknown type " + type);
+    }
+
+    /**
+     * Convert ColumnType to ColumnTypeChange.
+     *
+     * @param colType ColumnType.
+     * @param colTypeChg ColumnTypeChange to fullfill.
+     */
+    public static void convert(ColumnType colType, ColumnTypeChange colTypeChg) {
+        colTypeChg.changeType(colType.typeSpec().name());
+        // TODO varlen types
+    }
+
+    /**
+     * Convert ColumnTypeView to ColumnType.
+     *
+     * @param colTypeView ColumnTypeView.
+     * @return ColumnType.
+     */
+    public static ColumnType convert(ColumnTypeView colTypeView) {
+        String typeName = colTypeView.type();
+        ColumnType res = types.get(typeName);
+        if (res != null)
+            return res;
+        else
+            // TODO varlen types

Review comment:
       TODO with no ticket.

##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnChange;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexChange;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.schemas.table.TablesChange;
+import org.apache.ignite.configuration.tree.NamedListView;
+import org.apache.ignite.internal.schema.ColumnImpl;
+import org.apache.ignite.internal.schema.HashIndexImpl;
+import org.apache.ignite.internal.schema.PartialIndexImpl;
+import org.apache.ignite.internal.schema.PrimaryIndexImpl;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.SortedIndexColumnImpl;
+import org.apache.ignite.internal.schema.SortedIndexImpl;
+import org.apache.ignite.schema.Column;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.IndexColumn;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
+import org.apache.ignite.schema.SortedIndexColumn;
+import org.apache.ignite.schema.TableIndex;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/** Configuration to schema and vice versa converter. */
+public class SchemaConfigurationConverter {
+    /** Hash index type. */
+    private static final String HASH_TYPE = "HASH";
+
+    /** Sorted index type. */
+    private static final String SORTED_TYPE = "SORTED";
+
+    /** Partial index type. */
+    private static final String PARTIAL_TYPE = "PARTIAL";
+
+    /** Primary key index type. */
+    private static final String PK_TYPE = "PK";
+
+    /** Types map. */
+    private static Map<String, ColumnType> types = new HashMap<>();
+
+    static {
+        putType(ColumnType.INT8);
+        putType(ColumnType.INT16);
+        putType(ColumnType.INT32);
+        putType(ColumnType.INT64);
+        putType(ColumnType.UINT8);
+        putType(ColumnType.UINT16);
+        putType(ColumnType.UINT32);
+        putType(ColumnType.UINT64);
+        putType(ColumnType.FLOAT);
+        putType(ColumnType.DOUBLE);
+        putType(ColumnType.UUID);
+        putType(ColumnType.string());
+        putType(ColumnType.blobOf());
+        // TODO; handle length for some types
+        //putType(ColumnType.bitmaskOf());
+    }
+
+    private static void putType(ColumnType type) {
+        types.put(type.typeSpec().name(), type);
+    }
+
+    /**
+     * Convert SortedIndexColumn to IndexColumnChange.
+     * @param col IndexColumnChange.
+     * @param colInit IndexColumnChange to fullfill.
+     */
+    public static void convert(SortedIndexColumn col, IndexColumnChange colInit) {
+        colInit.changeName(col.name());
+        colInit.changeAsc(col.asc());
+    }
+
+    /**
+     * Convert IndexColumnView to SortedIndexColumn.
+     *
+     * @param colCfg IndexColumnView.
+     * @return SortedIndexColumn.
+     */
+    public static SortedIndexColumn convert(IndexColumnView colCfg) {
+        return new SortedIndexColumnImpl(colCfg.name(), colCfg.asc());
+    }
+
+    /**
+     * Convert TableIndex to TableIndexChange.
+     *
+     * @param idx TableIndex.
+     * @param idxChg TableIndexChange to fullfill.
+     */
+    public static void convert(TableIndex idx, TableIndexChange idxChg) {
+        idxChg.changeName(idx.name());
+        idxChg.changeType(idx.type());
+
+        if (HASH_TYPE.equals(idx.type())) {
+            HashIndex hashIdx = (HashIndex)idx;
+
+            String[] colNames = hashIdx.columns().stream().map(IndexColumn::name).toArray(String[]::new);
+            idxChg.changeColNames(colNames);
+        }
+        else if (PARTIAL_TYPE.equals(idx.type())) {
+            PartialIndex partIdx = (PartialIndex)idx;
+
+            idxChg.changeUniq(partIdx.unique());
+            idxChg.changeExpr(partIdx.expr());
+
+            idxChg.changeColumns(colsChg -> {
+                for (SortedIndexColumn col : partIdx.columns())
+                    colsChg.create(col.name(), colInit -> convert(col, colInit));
+            });
+
+        }
+        else if (SORTED_TYPE.equals(idx.type())) {
+            SortedIndex sortIdx = (SortedIndex)idx;
+            idxChg.changeUniq(sortIdx.unique());
+
+            idxChg.changeColumns(colsInit -> {
+                for (SortedIndexColumn col : sortIdx.columns())
+                    colsInit.create(col.name(), colInit -> convert(col, colInit));
+            });
+        }
+        else if (PK_TYPE.equals(idx.type())) {
+            PrimaryIndex primIdx = (PrimaryIndex)idx;
+
+            idxChg.changeColumns(colsInit -> {
+                for (SortedIndexColumn col : primIdx.columns())
+                    colsInit.create(col.name(), colInit -> convert(col, colInit));
+            });
+
+            idxChg.changeAffinityColumns(primIdx.affinityColumns().toArray(new String[primIdx.affinityColumns().size()]));
+        }
+        else throw new IllegalArgumentException("Unknown index type " + idx.type());
+    }
+
+    /**
+     * Convert TableIndexView into TableIndex.
+     *
+     * @param idxView TableIndexView.
+     * @return TableIndex.
+     */
+    public static TableIndex convert(TableIndexView idxView) {
+        String name = idxView.name();
+        String type = idxView.type();
+
+        if (type.equals("HASH")) {
+            String[] cols = idxView.colNames();
+
+            return new HashIndexImpl(name, cols);
+        }
+        else if (type.equals("SORTED") || type.equals("PARTIAL") || type.equals("PK")) {
+            if (type.equals("PARTIAL")) {
+                boolean uniq = idxView.uniq();
+                String expr = idxView.expr();
+                NamedListView<? extends IndexColumnView> colsView = idxView.columns();
+                List<SortedIndexColumn> cols = new ArrayList<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(colsView.get(key));
+                    cols.add(col);
+                }
+
+                return new PartialIndexImpl(name, cols, uniq, expr);
+            }
+            if (type.equals("SORTED")) {
+                boolean uniq = idxView.uniq();
+                List<SortedIndexColumn> cols = new ArrayList<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    cols.add(col);
+                }
+
+                return new SortedIndexImpl(name, cols, uniq);
+            }
+            if (type.equals("PK")) {
+                List<SortedIndexColumn> cols = new ArrayList<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    cols.add(col);
+                }
+                String[] affCols = idxView.colNames();
+
+                return new PrimaryIndexImpl(cols, Arrays.stream(affCols).collect(Collectors.toList()));
+            }
+
+            return null;
+        }
+        else
+            throw new IllegalArgumentException("Unknown type " + type);
+    }
+
+    /**
+     * Convert ColumnType to ColumnTypeChange.
+     *
+     * @param colType ColumnType.
+     * @param colTypeChg ColumnTypeChange to fullfill.
+     */
+    public static void convert(ColumnType colType, ColumnTypeChange colTypeChg) {
+        colTypeChg.changeType(colType.typeSpec().name());
+        // TODO varlen types
+    }
+
+    /**
+     * Convert ColumnTypeView to ColumnType.
+     *
+     * @param colTypeView ColumnTypeView.
+     * @return ColumnType.
+     */
+    public static ColumnType convert(ColumnTypeView colTypeView) {
+        String typeName = colTypeView.type();
+        ColumnType res = types.get(typeName);
+        if (res != null)
+            return res;
+        else
+            // TODO varlen types
+            throw new IllegalArgumentException();
+    }
+
+    /**
+     * Convert column to column change.
+     *
+     * @param col Column to convert.
+     * @param colChg Column
+     */
+    public static void convert(Column col, ColumnChange colChg) {
+        colChg.changeName(col.name());
+        colChg.changeType(colTypeInit -> convert(col.type(), colTypeInit));
+        if (col.defaultValue() != null)
+            colChg.changeDefaultValue(col.defaultValue().toString()); // TODO: specify "default type" type

Review comment:
       TODO with no ticket.




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

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



[GitHub] [ignite-3] AMashenkov commented on a change in pull request #132: GG-14290: SchemaDescriptor and ConfigurationSchema converters.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #132:
URL: https://github.com/apache/ignite-3/pull/132#discussion_r635034814



##########
File path: modules/schema/src/test/java/org/apache/ignite/internal/schema/configuration/SchemaDescriptorConverterTest.java
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.internal.schema.Column;
+import org.apache.ignite.internal.schema.Columns;
+import org.apache.ignite.internal.schema.NativeTypeSpec;
+import org.apache.ignite.internal.schema.SchemaDescriptor;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.SchemaBuilders;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.builder.SchemaTableBuilder;
+import org.apache.ignite.schema.builder.TableColumnBuilder;
+import org.junit.jupiter.api.Test;
+
+import java.util.function.Function;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/** Tests for SchemaDescriptorConverter. */
+public class SchemaDescriptorConverterTest {
+    /** Totoal number of columns. */
+    private static final int columns = 14;
+
+    /**
+     * Convert table with complex primary key and check it.
+     */
+    @Test
+    public void testComplexPrimaryIndex() {
+        SchemaTableBuilder bldr = getBuilder(false, false);
+        SchemaTable tblSchm = bldr.withIndex(SchemaBuilders.pkIndex()
+            .addIndexColumn("INT8").done()
+            .addIndexColumn("ID").done()
+            .build()
+        ).build();
+
+        SchemaDescriptor tblDscr = SchemaDescriptorConverter.convert(tblSchm);
+
+        assertEquals(2, tblSchm.keyColumns().size());
+        assertEquals(columns - 2, tblSchm.valueColumns().size());
+    }
+
+    /**
+     * Convert table with nullable columns.
+     */
+    @Test
+    public void convertNullable() {
+        testConvert(true);
+    }
+
+    /**
+     * Convert table with non nullable columns.
+     */
+    @Test
+    public void convertTypes() {
+        testConvert(false);
+    }
+
+    /**
+     * Test set of columns.
+     *
+     * @param nullable Nullable flag.
+     */
+    private void testConvert(boolean nullable) {
+        SchemaTable tblSchm = getBuilder(nullable, true).build();
+
+        SchemaDescriptor tblDscr = SchemaDescriptorConverter.convert(tblSchm);
+
+        assertEquals(1, tblDscr.keyColumns().length());
+        testCol(tblDscr.keyColumns(), "ID", NativeTypeSpec.UUID, nullable);
+
+        assertEquals(columns - 1, tblDscr.valueColumns().length());
+        testCol(tblDscr.valueColumns(), "INT8", NativeTypeSpec.BYTE, nullable);
+        testCol(tblDscr.valueColumns(), "INT16", NativeTypeSpec.SHORT, nullable);
+        testCol(tblDscr.valueColumns(), "INT32", NativeTypeSpec.INTEGER, nullable);
+        testCol(tblDscr.valueColumns(), "INT64", NativeTypeSpec.LONG, nullable);
+        testCol(tblDscr.valueColumns(), "FLOAT", NativeTypeSpec.FLOAT, nullable);
+        testCol(tblDscr.valueColumns(), "DOUBLE", NativeTypeSpec.DOUBLE, nullable);
+        testCol(tblDscr.valueColumns(), "UUID", NativeTypeSpec.UUID, nullable);
+        testCol(tblDscr.valueColumns(), "STRING", NativeTypeSpec.STRING, nullable);
+        testCol(tblDscr.valueColumns(), "STRING_FS10", NativeTypeSpec.STRING, nullable);
+        testCol(tblDscr.valueColumns(), "BLOB", NativeTypeSpec.BYTES, nullable);
+        testCol(tblDscr.valueColumns(), "DECIMAL", NativeTypeSpec.DECIMAL, nullable);
+        testCol(tblDscr.valueColumns(), "BITMASK_FS10", NativeTypeSpec.BITMASK, nullable);
+    }
+
+    /**
+     * Get SchemaTableBuilder with default table.
+     *
+     * @param nullable If all columns should be nullable.
+     * @param withPk If builder should contains primary key index.
+     * @return SchemaTableBuilder.
+     */
+    private SchemaTableBuilder getBuilder(boolean nullable, boolean withPk) {
+        Function<TableColumnBuilder, org.apache.ignite.schema.Column> postProcess = builder -> {
+            if (nullable)
+                builder.asNullable();
+            else
+                builder.asNonNull();
+            return builder.build();
+        };
+
+        SchemaTableBuilder res = SchemaBuilders.tableBuilder("SCHEMA", "TABLE")
+            .columns(
+                postProcess.apply(SchemaBuilders.column("ID", ColumnType.UUID)),
+                postProcess.apply(SchemaBuilders.column("INT8", ColumnType.INT8)),
+                postProcess.apply(SchemaBuilders.column("INT16", ColumnType.INT16)),
+                postProcess.apply(SchemaBuilders.column("INT32", ColumnType.INT32)),
+                postProcess.apply(SchemaBuilders.column("INT64", ColumnType.INT64)),
+                postProcess.apply(SchemaBuilders.column("FLOAT", ColumnType.FLOAT)),
+                postProcess.apply(SchemaBuilders.column("DOUBLE", ColumnType.DOUBLE)),
+                postProcess.apply(SchemaBuilders.column("UUID", ColumnType.UUID)),
+                postProcess.apply(SchemaBuilders.column("STRING", ColumnType.string())),
+                postProcess.apply(SchemaBuilders.column("STRING_FS10", ColumnType.stringOf(10))),
+                postProcess.apply(SchemaBuilders.column("BLOB", ColumnType.blobOf())),
+                postProcess.apply(SchemaBuilders.column("BLOB_FS10", ColumnType.blobOf(10))),
+                postProcess.apply(SchemaBuilders.column("DECIMAL", ColumnType.number(1,1))),
+                postProcess.apply(SchemaBuilders.column("BITMASK_FS10", ColumnType.bitmaskOf(10)))
+                // TODO: uncomment after unsigned types available

Review comment:
       TODO with no  ticket
   ```suggestion
                   // TODO: IGNITE-13750 uncomment after unsigned types available
   ```




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

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



[GitHub] [ignite-3] AMashenkov commented on a change in pull request #132: GG-14290: SchemaDescriptor and ConfigurationSchema converters.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #132:
URL: https://github.com/apache/ignite-3/pull/132#discussion_r635025250



##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaTableValidatorImpl.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.TableValidator;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.validation.ValidationContext;
+import org.apache.ignite.configuration.validation.ValidationIssue;
+import org.apache.ignite.configuration.validation.Validator;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.builder.SchemaTableBuilderImpl;
+import org.apache.ignite.schema.Column;
+
+import java.util.ArrayList;
+import java.util.Collection;
+
+/** SchemaTable validator implementation.  */
+public class SchemaTableValidatorImpl implements Validator<TableValidator, TableView> {
+    public final static SchemaTableValidatorImpl INSTANCE = new SchemaTableValidatorImpl();

Review comment:
       Javadocs




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

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



[GitHub] [ignite-3] AMashenkov commented on a change in pull request #132: GG-14290: SchemaDescriptor and ConfigurationSchema converters.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #132:
URL: https://github.com/apache/ignite-3/pull/132#discussion_r635027466



##########
File path: modules/schema/src/test/java/org/apache/ignite/internal/schema/builder/HashIndexBuilderTest.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.ignite.internal.schema.builder;
+
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.SchemaBuilders;
+import org.apache.ignite.schema.builder.HashIndexBuilder;
+import org.junit.jupiter.api.Test;
+
+import java.util.Collections;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+/** Tests for hash index builder. */
+public class HashIndexBuilderTest {
+    /** Full featured HashIndex. */
+    public static final HashIndex IDX;
+
+    static {
+        HashIndexBuilder builder = SchemaBuilders.hashIndex("testHI")
+            .withColumns("A", "B", "C")
+            .withHints(Collections.singletonMap("param","value"));
+        IDX = builder.build();
+    }
+
+    /**
+     * Build index and check its parameters.
+     */
+    @Test
+    public void testBuild() {
+

Review comment:
       ```suggestion
   ```

##########
File path: modules/schema/src/test/java/org/apache/ignite/internal/schema/builder/HashIndexBuilderTest.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.ignite.internal.schema.builder;
+
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.SchemaBuilders;
+import org.apache.ignite.schema.builder.HashIndexBuilder;
+import org.junit.jupiter.api.Test;
+
+import java.util.Collections;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+/** Tests for hash index builder. */
+public class HashIndexBuilderTest {
+    /** Full featured HashIndex. */
+    public static final HashIndex IDX;
+
+    static {
+        HashIndexBuilder builder = SchemaBuilders.hashIndex("testHI")
+            .withColumns("A", "B", "C")
+            .withHints(Collections.singletonMap("param","value"));
+        IDX = builder.build();

Review comment:
       ```suggestion
               .withHints(Collections.singletonMap("param","value"));
               
           IDX = builder.build();
   ```




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

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



[GitHub] [ignite-3] AMashenkov commented on a change in pull request #132: GG-14290: SchemaDescriptor and ConfigurationSchema converters.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #132:
URL: https://github.com/apache/ignite-3/pull/132#discussion_r635045587



##########
File path: modules/schema/src/test/java/org/apache/ignite/internal/schema/builder/HashIndexBuilderTest.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.ignite.internal.schema.builder;
+
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.SchemaBuilders;
+import org.apache.ignite.schema.builder.HashIndexBuilder;
+import org.junit.jupiter.api.Test;
+
+import java.util.Collections;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+/** Tests for hash index builder. */
+public class HashIndexBuilderTest {
+    /** Full featured HashIndex. */
+    public static final HashIndex IDX;

Review comment:
       What is the purpose of having a static field here?




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

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



[GitHub] [ignite-3] AMashenkov commented on a change in pull request #132: GG-14290: SchemaDescriptor and ConfigurationSchema converters.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #132:
URL: https://github.com/apache/ignite-3/pull/132#discussion_r635041760



##########
File path: modules/schema/src/test/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverterTest.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.ConfigurationRegistry;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableValidator;
+import org.apache.ignite.configuration.schemas.table.TablesConfiguration;
+import org.apache.ignite.internal.schema.builder.HashIndexBuilderTest;
+import org.apache.ignite.internal.schema.builder.PartialIndexBuilderTest;
+import org.apache.ignite.internal.schema.builder.PrimaryKeyBuilderTest;
+import org.apache.ignite.internal.schema.builder.SchemaTableBuilderTest;
+import org.apache.ignite.internal.schema.builder.SortedIndexBuilderTest;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Collections;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.ignite.internal.schema.builder.SchemaTableBuilderTest.TBL;
+
+/** SchmConfigurationConverter tests. */
+public class SchemaConfigurationConverterTest {
+    /** Configuration registry with one table for each test. */
+    private ConfigurationRegistry confRegistry;
+
+    /**
+     * Prepare configuration registry for test.
+     *
+     * @throws ExecutionException
+     * @throws InterruptedException
+     */
+    @BeforeEach
+    public void createRegistry() throws ExecutionException, InterruptedException {
+        confRegistry = new ConfigurationRegistry(
+            Collections.singleton(TablesConfiguration.KEY),
+            Collections.singletonMap(TableValidator.class, Collections.singleton(SchemaTableValidatorImpl.INSTANCE)),
+            Collections.singleton(new TestConfigurationStorage()));
+
+        confRegistry.getConfiguration(TablesConfiguration.KEY).change(
+            ch -> {
+                SchemaConfigurationConverter.createTable(TBL, ch);
+                ch.changeTables(tblsCh -> tblsCh.create(TBL.canonicalName(),
+                    tblCh -> tblCh.changeReplicas(1)));
+            }).get();
+    }
+
+    /**
+     * Add/remove HashIndex into configuration and read it back.
+     */
+    @Test
+    public void testConvertHashIndex() throws ExecutionException, InterruptedException {
+        HashIndex idx = HashIndexBuilderTest.IDX;
+        getTbl().change(ch -> SchemaConfigurationConverter.addIndex(idx, ch)).get();
+    }
+
+    /**
+     * Add/remove SortedIndex into configuration and read it back.
+     */
+    @Test
+    public void testConvertSortedIndex() throws ExecutionException, InterruptedException {
+        SortedIndex idx = SortedIndexBuilderTest.IDX;
+
+        getTbl().change(ch -> SchemaConfigurationConverter.addIndex(idx, ch)).get();
+    }
+
+    /**
+     * Add/remove PrimaryIndex into configuration and read it back.
+     */
+    @Test
+    public void testPKIndex() throws ExecutionException, InterruptedException {
+        PrimaryIndex idx = PrimaryKeyBuilderTest.IDX;
+
+        getTbl().change(ch -> SchemaConfigurationConverter.addIndex(idx, ch)).get();
+    }
+
+    /**
+     * Add/remove PartialIndex into configuration and read it back.
+     */
+    @Test
+    public void testPartialIndex() throws ExecutionException, InterruptedException {
+        PartialIndex idx = PartialIndexBuilderTest.IDX;
+
+        getTbl().change(ch -> SchemaConfigurationConverter.addIndex(idx, ch)).get();
+    }
+
+    /**
+     * Add/remove table and read it back.
+     */
+    @Test
+    public void testConvertTable() {
+        // TODO
+    }
+
+    /**
+     * Get tests default table configuration.
+     * @return Configuration of default table.

Review comment:
       ```suggestion
        * Get tests default table configuration.
        *
        * @return Configuration of default table.
   ```




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

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



[GitHub] [ignite-3] AMashenkov commented on a change in pull request #132: GG-14290: SchemaDescriptor and ConfigurationSchema converters.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #132:
URL: https://github.com/apache/ignite-3/pull/132#discussion_r636714797



##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnChange;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexChange;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.schemas.table.TablesChange;
+import org.apache.ignite.configuration.tree.NamedListView;
+import org.apache.ignite.internal.schema.ColumnImpl;
+import org.apache.ignite.internal.schema.HashIndexImpl;
+import org.apache.ignite.internal.schema.PartialIndexImpl;
+import org.apache.ignite.internal.schema.PrimaryIndexImpl;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.SortedIndexColumnImpl;
+import org.apache.ignite.internal.schema.SortedIndexImpl;
+import org.apache.ignite.schema.Column;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.IndexColumn;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
+import org.apache.ignite.schema.SortedIndexColumn;
+import org.apache.ignite.schema.TableIndex;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+/**
+ * Configuration to schema and vice versa converter.
+ */
+public class SchemaConfigurationConverter {
+    /** Hash index type. */
+    private static final String HASH_TYPE = "HASH";
+
+    /** Sorted index type. */
+    private static final String SORTED_TYPE = "SORTED";
+
+    /** Partial index type. */
+    private static final String PARTIAL_TYPE = "PARTIAL";
+
+    /** Primary key index type. */
+    private static final String PK_TYPE = "PK";
+
+    /** Types map. */
+    private static Map<String, ColumnType> types = new HashMap<>();
+
+    static {
+        putType(ColumnType.INT8);
+        putType(ColumnType.INT16);
+        putType(ColumnType.INT32);
+        putType(ColumnType.INT64);
+        putType(ColumnType.UINT8);
+        putType(ColumnType.UINT16);
+        putType(ColumnType.UINT32);
+        putType(ColumnType.UINT64);
+        putType(ColumnType.FLOAT);
+        putType(ColumnType.DOUBLE);
+        putType(ColumnType.UUID);
+    }
+
+    private static void putType(ColumnType type) {
+        types.put(type.typeSpec().name(), type);
+    }
+
+    /**
+     * Convert SortedIndexColumn to IndexColumnChange.
+     * @param col IndexColumnChange.
+     * @param colInit IndexColumnChange to fullfill.
+     */
+    public static void convert(SortedIndexColumn col, IndexColumnChange colInit) {
+        colInit.changeName(col.name());
+        colInit.changeAsc(col.asc());
+    }
+
+    /**
+     * Convert IndexColumnView to SortedIndexColumn.
+     *
+     * @param colCfg IndexColumnView.
+     * @return SortedIndexColumn.
+     */
+    public static SortedIndexColumn convert(IndexColumnView colCfg) {
+        return new SortedIndexColumnImpl(colCfg.name(), colCfg.asc());
+    }
+
+    /**
+     * Convert TableIndex to TableIndexChange.
+     *
+     * @param idx TableIndex.
+     * @param idxChg TableIndexChange to fullfill.
+     */
+    public static void convert(TableIndex idx, TableIndexChange idxChg) {
+        idxChg.changeName(idx.name());
+        idxChg.changeType(idx.type());
+        switch (idx.type()) {
+            case HASH_TYPE:
+                HashIndex hashIdx = (HashIndex)idx;
+
+                String[] colNames = hashIdx.columns().stream().map(IndexColumn::name).toArray(String[]::new);
+                idxChg.changeColNames(colNames);
+                break;
+            case PARTIAL_TYPE:
+                PartialIndex partIdx = (PartialIndex)idx;
+
+                idxChg.changeUniq(partIdx.unique());
+                idxChg.changeExpr(partIdx.expr());
+
+                idxChg.changeColumns(colsChg -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : partIdx.columns())

Review comment:
       ```suggestion
                       int colIdx = 0;
                       
                       for (SortedIndexColumn col : partIdx.columns())
   ```

##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnChange;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexChange;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.schemas.table.TablesChange;
+import org.apache.ignite.configuration.tree.NamedListView;
+import org.apache.ignite.internal.schema.ColumnImpl;
+import org.apache.ignite.internal.schema.HashIndexImpl;
+import org.apache.ignite.internal.schema.PartialIndexImpl;
+import org.apache.ignite.internal.schema.PrimaryIndexImpl;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.SortedIndexColumnImpl;
+import org.apache.ignite.internal.schema.SortedIndexImpl;
+import org.apache.ignite.schema.Column;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.IndexColumn;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
+import org.apache.ignite.schema.SortedIndexColumn;
+import org.apache.ignite.schema.TableIndex;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+/**
+ * Configuration to schema and vice versa converter.
+ */
+public class SchemaConfigurationConverter {
+    /** Hash index type. */
+    private static final String HASH_TYPE = "HASH";
+
+    /** Sorted index type. */
+    private static final String SORTED_TYPE = "SORTED";
+
+    /** Partial index type. */
+    private static final String PARTIAL_TYPE = "PARTIAL";
+
+    /** Primary key index type. */
+    private static final String PK_TYPE = "PK";
+
+    /** Types map. */
+    private static Map<String, ColumnType> types = new HashMap<>();
+
+    static {
+        putType(ColumnType.INT8);
+        putType(ColumnType.INT16);
+        putType(ColumnType.INT32);
+        putType(ColumnType.INT64);
+        putType(ColumnType.UINT8);
+        putType(ColumnType.UINT16);
+        putType(ColumnType.UINT32);
+        putType(ColumnType.UINT64);
+        putType(ColumnType.FLOAT);
+        putType(ColumnType.DOUBLE);
+        putType(ColumnType.UUID);
+    }
+
+    private static void putType(ColumnType type) {
+        types.put(type.typeSpec().name(), type);
+    }
+
+    /**
+     * Convert SortedIndexColumn to IndexColumnChange.
+     * @param col IndexColumnChange.
+     * @param colInit IndexColumnChange to fullfill.
+     */
+    public static void convert(SortedIndexColumn col, IndexColumnChange colInit) {
+        colInit.changeName(col.name());
+        colInit.changeAsc(col.asc());
+    }
+
+    /**
+     * Convert IndexColumnView to SortedIndexColumn.
+     *
+     * @param colCfg IndexColumnView.
+     * @return SortedIndexColumn.
+     */
+    public static SortedIndexColumn convert(IndexColumnView colCfg) {
+        return new SortedIndexColumnImpl(colCfg.name(), colCfg.asc());
+    }
+
+    /**
+     * Convert TableIndex to TableIndexChange.
+     *
+     * @param idx TableIndex.
+     * @param idxChg TableIndexChange to fullfill.
+     */
+    public static void convert(TableIndex idx, TableIndexChange idxChg) {
+        idxChg.changeName(idx.name());
+        idxChg.changeType(idx.type());
+        switch (idx.type()) {

Review comment:
       ```suggestion
           idxChg.changeType(idx.type());
           
           switch (idx.type()) {
   ```

##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnChange;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexChange;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.schemas.table.TablesChange;
+import org.apache.ignite.configuration.tree.NamedListView;
+import org.apache.ignite.internal.schema.ColumnImpl;
+import org.apache.ignite.internal.schema.HashIndexImpl;
+import org.apache.ignite.internal.schema.PartialIndexImpl;
+import org.apache.ignite.internal.schema.PrimaryIndexImpl;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.SortedIndexColumnImpl;
+import org.apache.ignite.internal.schema.SortedIndexImpl;
+import org.apache.ignite.schema.Column;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.IndexColumn;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
+import org.apache.ignite.schema.SortedIndexColumn;
+import org.apache.ignite.schema.TableIndex;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+/**
+ * Configuration to schema and vice versa converter.
+ */
+public class SchemaConfigurationConverter {
+    /** Hash index type. */
+    private static final String HASH_TYPE = "HASH";
+
+    /** Sorted index type. */
+    private static final String SORTED_TYPE = "SORTED";
+
+    /** Partial index type. */
+    private static final String PARTIAL_TYPE = "PARTIAL";
+
+    /** Primary key index type. */
+    private static final String PK_TYPE = "PK";
+
+    /** Types map. */
+    private static Map<String, ColumnType> types = new HashMap<>();
+
+    static {
+        putType(ColumnType.INT8);
+        putType(ColumnType.INT16);
+        putType(ColumnType.INT32);
+        putType(ColumnType.INT64);
+        putType(ColumnType.UINT8);
+        putType(ColumnType.UINT16);
+        putType(ColumnType.UINT32);
+        putType(ColumnType.UINT64);
+        putType(ColumnType.FLOAT);
+        putType(ColumnType.DOUBLE);
+        putType(ColumnType.UUID);
+    }
+
+    private static void putType(ColumnType type) {
+        types.put(type.typeSpec().name(), type);
+    }
+
+    /**
+     * Convert SortedIndexColumn to IndexColumnChange.
+     * @param col IndexColumnChange.
+     * @param colInit IndexColumnChange to fullfill.
+     */
+    public static void convert(SortedIndexColumn col, IndexColumnChange colInit) {
+        colInit.changeName(col.name());
+        colInit.changeAsc(col.asc());
+    }
+
+    /**
+     * Convert IndexColumnView to SortedIndexColumn.
+     *
+     * @param colCfg IndexColumnView.
+     * @return SortedIndexColumn.
+     */
+    public static SortedIndexColumn convert(IndexColumnView colCfg) {
+        return new SortedIndexColumnImpl(colCfg.name(), colCfg.asc());
+    }
+
+    /**
+     * Convert TableIndex to TableIndexChange.
+     *
+     * @param idx TableIndex.
+     * @param idxChg TableIndexChange to fullfill.
+     */
+    public static void convert(TableIndex idx, TableIndexChange idxChg) {
+        idxChg.changeName(idx.name());
+        idxChg.changeType(idx.type());
+        switch (idx.type()) {
+            case HASH_TYPE:
+                HashIndex hashIdx = (HashIndex)idx;
+
+                String[] colNames = hashIdx.columns().stream().map(IndexColumn::name).toArray(String[]::new);
+                idxChg.changeColNames(colNames);
+                break;
+            case PARTIAL_TYPE:
+                PartialIndex partIdx = (PartialIndex)idx;
+
+                idxChg.changeUniq(partIdx.unique());
+                idxChg.changeExpr(partIdx.expr());
+
+                idxChg.changeColumns(colsChg -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : partIdx.columns())
+                        colsChg.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+                break;
+            case SORTED_TYPE:
+                SortedIndex sortIdx = (SortedIndex)idx;
+                idxChg.changeUniq(sortIdx.unique());
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : sortIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));

Review comment:
       ```suggestion
                       int colIdx = 0;
                       
                       for (SortedIndexColumn col : sortIdx.columns())
                           colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
   ```

##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnChange;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexChange;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.schemas.table.TablesChange;
+import org.apache.ignite.configuration.tree.NamedListView;
+import org.apache.ignite.internal.schema.ColumnImpl;
+import org.apache.ignite.internal.schema.HashIndexImpl;
+import org.apache.ignite.internal.schema.PartialIndexImpl;
+import org.apache.ignite.internal.schema.PrimaryIndexImpl;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.SortedIndexColumnImpl;
+import org.apache.ignite.internal.schema.SortedIndexImpl;
+import org.apache.ignite.schema.Column;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.IndexColumn;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
+import org.apache.ignite.schema.SortedIndexColumn;
+import org.apache.ignite.schema.TableIndex;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+/**
+ * Configuration to schema and vice versa converter.
+ */
+public class SchemaConfigurationConverter {
+    /** Hash index type. */
+    private static final String HASH_TYPE = "HASH";
+
+    /** Sorted index type. */
+    private static final String SORTED_TYPE = "SORTED";
+
+    /** Partial index type. */
+    private static final String PARTIAL_TYPE = "PARTIAL";
+
+    /** Primary key index type. */
+    private static final String PK_TYPE = "PK";
+
+    /** Types map. */
+    private static Map<String, ColumnType> types = new HashMap<>();
+
+    static {
+        putType(ColumnType.INT8);
+        putType(ColumnType.INT16);
+        putType(ColumnType.INT32);
+        putType(ColumnType.INT64);
+        putType(ColumnType.UINT8);
+        putType(ColumnType.UINT16);
+        putType(ColumnType.UINT32);
+        putType(ColumnType.UINT64);
+        putType(ColumnType.FLOAT);
+        putType(ColumnType.DOUBLE);
+        putType(ColumnType.UUID);
+    }
+
+    private static void putType(ColumnType type) {
+        types.put(type.typeSpec().name(), type);
+    }
+
+    /**
+     * Convert SortedIndexColumn to IndexColumnChange.
+     * @param col IndexColumnChange.
+     * @param colInit IndexColumnChange to fullfill.
+     */
+    public static void convert(SortedIndexColumn col, IndexColumnChange colInit) {
+        colInit.changeName(col.name());
+        colInit.changeAsc(col.asc());
+    }
+
+    /**
+     * Convert IndexColumnView to SortedIndexColumn.
+     *
+     * @param colCfg IndexColumnView.
+     * @return SortedIndexColumn.
+     */
+    public static SortedIndexColumn convert(IndexColumnView colCfg) {
+        return new SortedIndexColumnImpl(colCfg.name(), colCfg.asc());
+    }
+
+    /**
+     * Convert TableIndex to TableIndexChange.
+     *
+     * @param idx TableIndex.
+     * @param idxChg TableIndexChange to fullfill.
+     */
+    public static void convert(TableIndex idx, TableIndexChange idxChg) {
+        idxChg.changeName(idx.name());
+        idxChg.changeType(idx.type());
+        switch (idx.type()) {
+            case HASH_TYPE:
+                HashIndex hashIdx = (HashIndex)idx;
+
+                String[] colNames = hashIdx.columns().stream().map(IndexColumn::name).toArray(String[]::new);
+                idxChg.changeColNames(colNames);
+                break;
+            case PARTIAL_TYPE:
+                PartialIndex partIdx = (PartialIndex)idx;
+
+                idxChg.changeUniq(partIdx.unique());
+                idxChg.changeExpr(partIdx.expr());
+
+                idxChg.changeColumns(colsChg -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : partIdx.columns())
+                        colsChg.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+                break;

Review comment:
       ```suggestion
                   });
                   
                   break;
   ```

##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnChange;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexChange;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.schemas.table.TablesChange;
+import org.apache.ignite.configuration.tree.NamedListView;
+import org.apache.ignite.internal.schema.ColumnImpl;
+import org.apache.ignite.internal.schema.HashIndexImpl;
+import org.apache.ignite.internal.schema.PartialIndexImpl;
+import org.apache.ignite.internal.schema.PrimaryIndexImpl;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.SortedIndexColumnImpl;
+import org.apache.ignite.internal.schema.SortedIndexImpl;
+import org.apache.ignite.schema.Column;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.IndexColumn;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
+import org.apache.ignite.schema.SortedIndexColumn;
+import org.apache.ignite.schema.TableIndex;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+/**
+ * Configuration to schema and vice versa converter.
+ */
+public class SchemaConfigurationConverter {
+    /** Hash index type. */
+    private static final String HASH_TYPE = "HASH";
+
+    /** Sorted index type. */
+    private static final String SORTED_TYPE = "SORTED";
+
+    /** Partial index type. */
+    private static final String PARTIAL_TYPE = "PARTIAL";
+
+    /** Primary key index type. */
+    private static final String PK_TYPE = "PK";
+
+    /** Types map. */
+    private static Map<String, ColumnType> types = new HashMap<>();
+
+    static {
+        putType(ColumnType.INT8);
+        putType(ColumnType.INT16);
+        putType(ColumnType.INT32);
+        putType(ColumnType.INT64);
+        putType(ColumnType.UINT8);
+        putType(ColumnType.UINT16);
+        putType(ColumnType.UINT32);
+        putType(ColumnType.UINT64);
+        putType(ColumnType.FLOAT);
+        putType(ColumnType.DOUBLE);
+        putType(ColumnType.UUID);
+    }
+
+    private static void putType(ColumnType type) {
+        types.put(type.typeSpec().name(), type);
+    }
+
+    /**
+     * Convert SortedIndexColumn to IndexColumnChange.
+     * @param col IndexColumnChange.
+     * @param colInit IndexColumnChange to fullfill.
+     */
+    public static void convert(SortedIndexColumn col, IndexColumnChange colInit) {
+        colInit.changeName(col.name());
+        colInit.changeAsc(col.asc());
+    }
+
+    /**
+     * Convert IndexColumnView to SortedIndexColumn.
+     *
+     * @param colCfg IndexColumnView.
+     * @return SortedIndexColumn.
+     */
+    public static SortedIndexColumn convert(IndexColumnView colCfg) {
+        return new SortedIndexColumnImpl(colCfg.name(), colCfg.asc());
+    }
+
+    /**
+     * Convert TableIndex to TableIndexChange.
+     *
+     * @param idx TableIndex.
+     * @param idxChg TableIndexChange to fullfill.
+     */
+    public static void convert(TableIndex idx, TableIndexChange idxChg) {
+        idxChg.changeName(idx.name());
+        idxChg.changeType(idx.type());
+        switch (idx.type()) {
+            case HASH_TYPE:
+                HashIndex hashIdx = (HashIndex)idx;
+
+                String[] colNames = hashIdx.columns().stream().map(IndexColumn::name).toArray(String[]::new);
+                idxChg.changeColNames(colNames);
+                break;
+            case PARTIAL_TYPE:
+                PartialIndex partIdx = (PartialIndex)idx;
+
+                idxChg.changeUniq(partIdx.unique());
+                idxChg.changeExpr(partIdx.expr());
+
+                idxChg.changeColumns(colsChg -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : partIdx.columns())
+                        colsChg.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+                break;
+            case SORTED_TYPE:
+                SortedIndex sortIdx = (SortedIndex)idx;
+                idxChg.changeUniq(sortIdx.unique());
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : sortIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+                break;
+            case PK_TYPE:
+                PrimaryIndex primIdx = (PrimaryIndex)idx;
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : primIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+
+                idxChg.changeAffinityColumns(primIdx.affinityColumns().toArray(new String[primIdx.affinityColumns().size()]));
+                break;
+            default:
+                throw new IllegalArgumentException("Unknown index type " + idx.type());
+        }
+    }
+
+    /**
+     * Convert TableIndexView into TableIndex.
+     *
+     * @param idxView TableIndexView.
+     * @return TableIndex.
+     */
+    public static TableIndex convert(TableIndexView idxView) {
+        String name = idxView.name();
+        String type = idxView.type();
+        switch (type) {
+            case "HASH":
+                String[] hashCols = idxView.colNames();
+
+                return new HashIndexImpl(name, hashCols);
+
+            case "SORTED":
+                boolean sortedUniq = idxView.uniq();
+                SortedMap<Integer, SortedIndexColumn> sortedCols = new TreeMap();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    sortedCols.put(Integer.valueOf(key), col);
+                }
+
+                return new SortedIndexImpl(name, new ArrayList<>(sortedCols.values()), sortedUniq);
+
+            case "PARTIAL":
+                boolean partialUniq = idxView.uniq();
+                String expr = idxView.expr();
+                NamedListView<? extends IndexColumnView> colsView = idxView.columns();
+                SortedMap<Integer, SortedIndexColumn> partialCols = new TreeMap<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(colsView.get(key));
+                    partialCols.put(Integer.valueOf(key), col);
+                }
+
+                return new PartialIndexImpl(name, new ArrayList<>(partialCols.values()), partialUniq, expr);
+
+            case "PK":
+                SortedMap<Integer, SortedIndexColumn> cols = new TreeMap<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    cols.put(Integer.valueOf(key), col);
+                }
+                String[] affCols = idxView.affinityColumns();
+
+                return new PrimaryIndexImpl(new ArrayList(cols.values()), List.of(affCols));
+
+            default:
+                throw new IllegalArgumentException("Unknown type " + type);
+        }
+    }
+
+    /**
+     * Convert ColumnType to ColumnTypeChange.
+     *
+     * @param colType ColumnType.
+     * @param colTypeChg ColumnTypeChange to fullfill.
+     */
+    public static void convert(ColumnType colType, ColumnTypeChange colTypeChg) {
+        String typeName = colType.typeSpec().name();
+        if (types.containsKey(typeName))
+            colTypeChg.changeType(typeName);
+        else {
+            colTypeChg.changeType(typeName);
+            switch (typeName) {
+                case "BITMASK":
+                case "BLOB":
+                case "STRING":
+                    ColumnType.VarLenColumnType varLenColType = (ColumnType.VarLenColumnType)colType;
+
+                    colTypeChg.changeLength(varLenColType.length());
+                    break;
+
+                case "DECIMAL":
+                    ColumnType.NumericColumnType numColType = (ColumnType.NumericColumnType)colType;
+
+                    colTypeChg.changePrecision(numColType.precision());
+                    colTypeChg.changeScale(numColType.scale());
+                    break;
+                default:
+                    throw new IllegalArgumentException("Unknown type " + colType.typeSpec().name());
+            }
+        }
+    }
+
+    /**
+     * Convert ColumnTypeView to ColumnType.
+     *
+     * @param colTypeView ColumnTypeView.
+     * @return ColumnType.
+     */
+    public static ColumnType convert(ColumnTypeView colTypeView) {
+        String typeName = colTypeView.type();
+        ColumnType res = types.get(typeName);
+        if (res != null)
+            return res;
+        else {
+            switch (typeName) {
+                case "BITMASK":
+                    int bitmaskLen = colTypeView.length();
+
+                    return ColumnType.bitmaskOf(bitmaskLen);
+
+                case "STRING":
+                    int strLen = colTypeView.length();
+
+                    return ColumnType.stringOf(strLen);
+
+                case "BLOB":
+                    int blobLen = colTypeView.length();
+
+                    return ColumnType.blobOf(blobLen);
+
+                case "DECIMAL":
+                    int prec = colTypeView.precision();
+                    int scale = colTypeView.scale();
+
+                    return ColumnType.number(prec, scale);
+
+                default:
+                    throw new IllegalArgumentException("Unknown type " + typeName);
+            }
+        }
+    }
+
+    /**
+     * Convert column to column change.
+     *
+     * @param col Column to convert.
+     * @param colChg Column
+     */
+    public static void convert(Column col, ColumnChange colChg) {
+        colChg.changeName(col.name());
+        colChg.changeType(colTypeInit -> convert(col.type(), colTypeInit));
+        if (col.defaultValue() != null)
+            colChg.changeDefaultValue(col.defaultValue().toString());
+        colChg.changeNullable(col.nullable());
+    }
+
+    /**
+     * Convert column view to Column.
+     *
+     * @param colView Column view.
+     * @return Column.
+     */
+    public static Column convert(ColumnView colView) {
+        String name = colView.name();
+        ColumnType type = convert(colView.type());
+        boolean nullable = colView.nullable();
+        String defValue = colView.defaultValue();
+
+        return new ColumnImpl(name, type, nullable, defValue);
+    }
+
+    /**
+     * Convert schema table to schema table change.
+     *
+     * @param tbl Schema table to convert.
+     * @param tblChg Change to fullfill.
+     */
+    public static void convert(SchemaTable tbl, TableChange tblChg) {
+        tblChg.changeName(tbl.canonicalName());
+        tblChg.changeIndices(idxsChg -> {
+            int idxIdx = 0;
+            for (TableIndex idx : tbl.indices())
+                idxsChg.create(String.valueOf(idxIdx++), idxInit -> convert(idx, idxInit));
+        });
+        tblChg.changeColumns(colsChg -> {
+            int colIdx = 0;
+            for (Column col : tbl.keyColumns())
+                colsChg.create(String.valueOf(colIdx++), colChg -> convert(col, colChg));
+            for (Column col : tbl.valueColumns())
+                colsChg.create(String.valueOf(colIdx++), colChg -> convert(col, colChg));
+        });

Review comment:
       ```suggestion
           tblChg.changeName(tbl.canonicalName());
           
           tblChg.changeIndices(idxsChg -> {
               int idxIdx = 0;
               
               for (TableIndex idx : tbl.indices())
                   idxsChg.create(String.valueOf(idxIdx++), idxInit -> convert(idx, idxInit));
           });
           tblChg.changeColumns(colsChg -> {
               int colIdx = 0;
               
               for (Column col : tbl.keyColumns())
                   colsChg.create(String.valueOf(colIdx++), colChg -> convert(col, colChg));
                   
               for (Column col : tbl.valueColumns())
                   colsChg.create(String.valueOf(colIdx++), colChg -> convert(col, colChg));
           });
   ```

##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnChange;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexChange;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.schemas.table.TablesChange;
+import org.apache.ignite.configuration.tree.NamedListView;
+import org.apache.ignite.internal.schema.ColumnImpl;
+import org.apache.ignite.internal.schema.HashIndexImpl;
+import org.apache.ignite.internal.schema.PartialIndexImpl;
+import org.apache.ignite.internal.schema.PrimaryIndexImpl;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.SortedIndexColumnImpl;
+import org.apache.ignite.internal.schema.SortedIndexImpl;
+import org.apache.ignite.schema.Column;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.IndexColumn;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
+import org.apache.ignite.schema.SortedIndexColumn;
+import org.apache.ignite.schema.TableIndex;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+/**
+ * Configuration to schema and vice versa converter.
+ */
+public class SchemaConfigurationConverter {
+    /** Hash index type. */
+    private static final String HASH_TYPE = "HASH";
+
+    /** Sorted index type. */
+    private static final String SORTED_TYPE = "SORTED";
+
+    /** Partial index type. */
+    private static final String PARTIAL_TYPE = "PARTIAL";
+
+    /** Primary key index type. */
+    private static final String PK_TYPE = "PK";
+
+    /** Types map. */
+    private static Map<String, ColumnType> types = new HashMap<>();
+
+    static {
+        putType(ColumnType.INT8);
+        putType(ColumnType.INT16);
+        putType(ColumnType.INT32);
+        putType(ColumnType.INT64);
+        putType(ColumnType.UINT8);
+        putType(ColumnType.UINT16);
+        putType(ColumnType.UINT32);
+        putType(ColumnType.UINT64);
+        putType(ColumnType.FLOAT);
+        putType(ColumnType.DOUBLE);
+        putType(ColumnType.UUID);
+    }
+
+    private static void putType(ColumnType type) {
+        types.put(type.typeSpec().name(), type);
+    }
+
+    /**
+     * Convert SortedIndexColumn to IndexColumnChange.
+     * @param col IndexColumnChange.
+     * @param colInit IndexColumnChange to fullfill.
+     */
+    public static void convert(SortedIndexColumn col, IndexColumnChange colInit) {
+        colInit.changeName(col.name());
+        colInit.changeAsc(col.asc());
+    }
+
+    /**
+     * Convert IndexColumnView to SortedIndexColumn.
+     *
+     * @param colCfg IndexColumnView.
+     * @return SortedIndexColumn.
+     */
+    public static SortedIndexColumn convert(IndexColumnView colCfg) {
+        return new SortedIndexColumnImpl(colCfg.name(), colCfg.asc());
+    }
+
+    /**
+     * Convert TableIndex to TableIndexChange.
+     *
+     * @param idx TableIndex.
+     * @param idxChg TableIndexChange to fullfill.
+     */
+    public static void convert(TableIndex idx, TableIndexChange idxChg) {
+        idxChg.changeName(idx.name());
+        idxChg.changeType(idx.type());
+        switch (idx.type()) {
+            case HASH_TYPE:
+                HashIndex hashIdx = (HashIndex)idx;
+
+                String[] colNames = hashIdx.columns().stream().map(IndexColumn::name).toArray(String[]::new);
+                idxChg.changeColNames(colNames);
+                break;
+            case PARTIAL_TYPE:
+                PartialIndex partIdx = (PartialIndex)idx;
+
+                idxChg.changeUniq(partIdx.unique());
+                idxChg.changeExpr(partIdx.expr());
+
+                idxChg.changeColumns(colsChg -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : partIdx.columns())
+                        colsChg.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+                break;
+            case SORTED_TYPE:
+                SortedIndex sortIdx = (SortedIndex)idx;
+                idxChg.changeUniq(sortIdx.unique());
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : sortIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+                break;
+            case PK_TYPE:
+                PrimaryIndex primIdx = (PrimaryIndex)idx;
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : primIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+
+                idxChg.changeAffinityColumns(primIdx.affinityColumns().toArray(new String[primIdx.affinityColumns().size()]));
+                break;
+            default:
+                throw new IllegalArgumentException("Unknown index type " + idx.type());
+        }
+    }
+
+    /**
+     * Convert TableIndexView into TableIndex.
+     *
+     * @param idxView TableIndexView.
+     * @return TableIndex.
+     */
+    public static TableIndex convert(TableIndexView idxView) {
+        String name = idxView.name();
+        String type = idxView.type();
+        switch (type) {
+            case "HASH":
+                String[] hashCols = idxView.colNames();
+
+                return new HashIndexImpl(name, hashCols);
+
+            case "SORTED":
+                boolean sortedUniq = idxView.uniq();
+                SortedMap<Integer, SortedIndexColumn> sortedCols = new TreeMap();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    sortedCols.put(Integer.valueOf(key), col);
+                }
+
+                return new SortedIndexImpl(name, new ArrayList<>(sortedCols.values()), sortedUniq);
+
+            case "PARTIAL":
+                boolean partialUniq = idxView.uniq();
+                String expr = idxView.expr();
+                NamedListView<? extends IndexColumnView> colsView = idxView.columns();
+                SortedMap<Integer, SortedIndexColumn> partialCols = new TreeMap<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(colsView.get(key));
+                    partialCols.put(Integer.valueOf(key), col);
+                }
+
+                return new PartialIndexImpl(name, new ArrayList<>(partialCols.values()), partialUniq, expr);
+
+            case "PK":
+                SortedMap<Integer, SortedIndexColumn> cols = new TreeMap<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    cols.put(Integer.valueOf(key), col);
+                }
+                String[] affCols = idxView.affinityColumns();
+
+                return new PrimaryIndexImpl(new ArrayList(cols.values()), List.of(affCols));
+
+            default:
+                throw new IllegalArgumentException("Unknown type " + type);
+        }
+    }
+
+    /**
+     * Convert ColumnType to ColumnTypeChange.
+     *
+     * @param colType ColumnType.
+     * @param colTypeChg ColumnTypeChange to fullfill.
+     */
+    public static void convert(ColumnType colType, ColumnTypeChange colTypeChg) {
+        String typeName = colType.typeSpec().name();
+        if (types.containsKey(typeName))
+            colTypeChg.changeType(typeName);
+        else {
+            colTypeChg.changeType(typeName);
+            switch (typeName) {
+                case "BITMASK":
+                case "BLOB":
+                case "STRING":
+                    ColumnType.VarLenColumnType varLenColType = (ColumnType.VarLenColumnType)colType;
+
+                    colTypeChg.changeLength(varLenColType.length());
+                    break;
+
+                case "DECIMAL":
+                    ColumnType.NumericColumnType numColType = (ColumnType.NumericColumnType)colType;
+
+                    colTypeChg.changePrecision(numColType.precision());
+                    colTypeChg.changeScale(numColType.scale());
+                    break;
+                default:
+                    throw new IllegalArgumentException("Unknown type " + colType.typeSpec().name());
+            }
+        }
+    }
+
+    /**
+     * Convert ColumnTypeView to ColumnType.
+     *
+     * @param colTypeView ColumnTypeView.
+     * @return ColumnType.
+     */
+    public static ColumnType convert(ColumnTypeView colTypeView) {
+        String typeName = colTypeView.type();
+        ColumnType res = types.get(typeName);
+        if (res != null)
+            return res;
+        else {
+            switch (typeName) {
+                case "BITMASK":
+                    int bitmaskLen = colTypeView.length();
+
+                    return ColumnType.bitmaskOf(bitmaskLen);
+
+                case "STRING":
+                    int strLen = colTypeView.length();
+
+                    return ColumnType.stringOf(strLen);
+
+                case "BLOB":
+                    int blobLen = colTypeView.length();
+
+                    return ColumnType.blobOf(blobLen);
+
+                case "DECIMAL":
+                    int prec = colTypeView.precision();
+                    int scale = colTypeView.scale();
+
+                    return ColumnType.number(prec, scale);
+
+                default:
+                    throw new IllegalArgumentException("Unknown type " + typeName);
+            }
+        }
+    }
+
+    /**
+     * Convert column to column change.
+     *
+     * @param col Column to convert.
+     * @param colChg Column
+     */
+    public static void convert(Column col, ColumnChange colChg) {
+        colChg.changeName(col.name());
+        colChg.changeType(colTypeInit -> convert(col.type(), colTypeInit));
+        if (col.defaultValue() != null)
+            colChg.changeDefaultValue(col.defaultValue().toString());
+        colChg.changeNullable(col.nullable());
+    }
+
+    /**
+     * Convert column view to Column.
+     *
+     * @param colView Column view.
+     * @return Column.
+     */
+    public static Column convert(ColumnView colView) {
+        String name = colView.name();
+        ColumnType type = convert(colView.type());
+        boolean nullable = colView.nullable();
+        String defValue = colView.defaultValue();
+
+        return new ColumnImpl(name, type, nullable, defValue);
+    }
+
+    /**
+     * Convert schema table to schema table change.
+     *
+     * @param tbl Schema table to convert.
+     * @param tblChg Change to fullfill.
+     */
+    public static void convert(SchemaTable tbl, TableChange tblChg) {
+        tblChg.changeName(tbl.canonicalName());
+        tblChg.changeIndices(idxsChg -> {
+            int idxIdx = 0;
+            for (TableIndex idx : tbl.indices())
+                idxsChg.create(String.valueOf(idxIdx++), idxInit -> convert(idx, idxInit));
+        });
+        tblChg.changeColumns(colsChg -> {
+            int colIdx = 0;
+            for (Column col : tbl.keyColumns())
+                colsChg.create(String.valueOf(colIdx++), colChg -> convert(col, colChg));
+            for (Column col : tbl.valueColumns())
+                colsChg.create(String.valueOf(colIdx++), colChg -> convert(col, colChg));
+        });
+    }
+
+    /**
+     * Convert TableConfiguration to SchemaTable.
+     *
+     * @param tblCfg TableConfiguration to convert.
+     * @return SchemaTable.
+     */
+    public static SchemaTable convert(TableConfiguration tblCfg) {
+        return convert(tblCfg.value());
+    }
+
+    /**
+     * Convert configuration to SchemaTable.
+     *
+     * @param tblView TableView to convert.
+     * @return SchemaTable.
+     */
+    public static SchemaTableImpl convert(TableView tblView) {
+        String canonicalName = tblView.name();
+
+        NamedListView<? extends ColumnView> colsView = tblView.columns();
+        SortedMap<Integer, Column> columns = new TreeMap<>();
+        for (String key : colsView.namedListKeys()) {
+            ColumnView colView = colsView.get(key);
+            Column col = convert(colView);
+            columns.put(Integer.valueOf(key), col);
+        }

Review comment:
       ```suggestion
   
           NamedListView<? extends ColumnView> colsView = tblView.columns();
           
           SortedMap<Integer, Column> columns = new TreeMap<>();
           
           for (String key : colsView.namedListKeys()) {
               Column col = convert(colsView.get(key));
               
               columns.put(Integer.valueOf(key), col);
           }
   ```

##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaDescriptorConverter.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.internal.schema.Column;
+import org.apache.ignite.internal.schema.InvalidTypeException;
+import org.apache.ignite.internal.schema.NativeType;
+import org.apache.ignite.internal.schema.NativeTypes;
+import org.apache.ignite.internal.schema.SchemaDescriptor;
+
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.SchemaTable;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+
+import static org.apache.ignite.internal.schema.NativeTypes.BYTE;
+import static org.apache.ignite.internal.schema.NativeTypes.DOUBLE;
+import static org.apache.ignite.internal.schema.NativeTypes.FLOAT;
+import static org.apache.ignite.internal.schema.NativeTypes.INTEGER;
+import static org.apache.ignite.internal.schema.NativeTypes.LONG;
+import static org.apache.ignite.internal.schema.NativeTypes.SHORT;
+import static org.apache.ignite.internal.schema.NativeTypes.UUID;
+
+/**
+ * Build SchemaDescriptor from SchemaTable internal configuration.
+ */
+public class SchemaDescriptorConverter {
+    /**
+     * Convert ColumnType to NativeType.
+     *
+     * @param colType ColumnType.
+     * @return NativeType.
+     */
+    private static NativeType convert(ColumnType colType) {
+        assert colType != null;
+
+        ColumnType.ColumnTypeSpec type = colType.typeSpec();
+        switch (type) {
+            case INT8:
+                return BYTE;
+
+            case INT16:
+                return SHORT;
+
+            case INT32:
+                return INTEGER;
+
+            case INT64:
+                return LONG;
+
+            case UINT8:
+            case UINT16:
+            case UINT32:
+            case UINT64:
+                throw new UnsupportedOperationException("Unsigned types are not supported yet.");
+
+            case FLOAT:
+                return FLOAT;
+
+            case DOUBLE:
+                return DOUBLE;
+
+            case DECIMAL:
+                ColumnType.NumericColumnType numType = (ColumnType.NumericColumnType)colType;
+
+                return NativeTypes.decimalOf(numType.precision(), numType.scale());
+            case UUID:
+                return UUID;
+
+            case BITMASK:
+                return NativeTypes.bitmaskOf(((ColumnType.VarLenColumnType) colType).length());
+
+            case STRING:
+                return NativeTypes.stringOf(((ColumnType.VarLenColumnType)colType).length());
+
+            case BLOB:
+                return NativeTypes.blobOf(((ColumnType.VarLenColumnType)colType).length());
+
+                default:
+                throw new InvalidTypeException("Unexpected type " + type);
+        }
+    }
+
+    /**
+     * Convert column from public configuration to internal.
+     *
+     * @param colCfg Column to confvert.
+     * @return Internal Column.
+     */
+    private static Column convert(org.apache.ignite.schema.Column colCfg) {
+        return new Column(colCfg.name(), convert(colCfg.type()), colCfg.nullable());
+    }
+
+    /**
+     * Build schema descriptor by SchemaTable.
+     *
+     * @param tblId Table id.
+     * @param schemaVer Schema version.
+     * @param tblCfg SchemaTable.
+     * @return SchemaDescriptor.
+     */
+    public static SchemaDescriptor convert(UUID tblId, int schemaVer, SchemaTable tblCfg) {
+        List<org.apache.ignite.schema.Column> keyColsCfg = new ArrayList<>(tblCfg.keyColumns());
+        Column[] keyCols = new Column[keyColsCfg.size()];
+        for (int i = 0;i < keyCols.length;i++)
+            keyCols[i] = convert(keyColsCfg.get(i));
+
+        String[] affCols = tblCfg.affinityColumns().stream().map(org.apache.ignite.schema.Column::name)
+            .toArray(String[]::new);
+
+        List<org.apache.ignite.schema.Column> valColsCfg = new ArrayList<>(tblCfg.valueColumns());
+        Column[] valCols = new Column[valColsCfg.size()];
+        for (int i = 0;i < valCols.length;i++)

Review comment:
       ```suggestion
           List<org.apache.ignite.schema.Column> valColsCfg = new ArrayList<>(tblCfg.valueColumns());
           
           Column[] valCols = new Column[valColsCfg.size()];
           
           for (int i = 0;i < valCols.length;i++)
   ```

##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaTableValidatorImpl.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.TableValidator;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.tree.NamedListView;
+import org.apache.ignite.configuration.validation.ValidationContext;
+import org.apache.ignite.configuration.validation.ValidationIssue;
+import org.apache.ignite.configuration.validation.Validator;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.builder.SchemaTableBuilderImpl;
+import org.apache.ignite.schema.Column;
+
+import java.util.ArrayList;
+import java.util.Collection;
+
+/**
+ * SchemaTable validator implementation.
+ */
+public class SchemaTableValidatorImpl implements Validator<TableValidator, NamedListView<TableView>> {
+    public final static SchemaTableValidatorImpl INSTANCE = new SchemaTableValidatorImpl();
+
+    /** {@inheritDoc} */
+    @Override public void validate(TableValidator annotation, ValidationContext<NamedListView<TableView>> ctx) {
+        NamedListView<TableView> list = ctx.getNewValue();
+        for (String key : list.namedListKeys()) {
+            TableView view = list.get(key);
+            try {
+                SchemaTableImpl tbl = SchemaConfigurationConverter.convert(view);
+                Collection<Column> allColumns = new ArrayList<>(tbl.keyColumns());
+                allColumns.addAll(tbl.valueColumns());
+                SchemaTableBuilderImpl.validateIndices(tbl.indices(), allColumns);
+            }

Review comment:
       ```suggestion
   
               try {
                   SchemaTableImpl tbl = SchemaConfigurationConverter.convert(view);
                   
                   Collection<Column> allColumns = new ArrayList<>(tbl.keyColumns());
                   
                   allColumns.addAll(tbl.valueColumns());
                   
                   SchemaTableBuilderImpl.validateIndices(tbl.indices(), allColumns);
               }
   ```

##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnChange;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexChange;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.schemas.table.TablesChange;
+import org.apache.ignite.configuration.tree.NamedListView;
+import org.apache.ignite.internal.schema.ColumnImpl;
+import org.apache.ignite.internal.schema.HashIndexImpl;
+import org.apache.ignite.internal.schema.PartialIndexImpl;
+import org.apache.ignite.internal.schema.PrimaryIndexImpl;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.SortedIndexColumnImpl;
+import org.apache.ignite.internal.schema.SortedIndexImpl;
+import org.apache.ignite.schema.Column;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.IndexColumn;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
+import org.apache.ignite.schema.SortedIndexColumn;
+import org.apache.ignite.schema.TableIndex;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+/**
+ * Configuration to schema and vice versa converter.
+ */
+public class SchemaConfigurationConverter {
+    /** Hash index type. */
+    private static final String HASH_TYPE = "HASH";
+
+    /** Sorted index type. */
+    private static final String SORTED_TYPE = "SORTED";
+
+    /** Partial index type. */
+    private static final String PARTIAL_TYPE = "PARTIAL";
+
+    /** Primary key index type. */
+    private static final String PK_TYPE = "PK";
+
+    /** Types map. */
+    private static Map<String, ColumnType> types = new HashMap<>();
+
+    static {
+        putType(ColumnType.INT8);
+        putType(ColumnType.INT16);
+        putType(ColumnType.INT32);
+        putType(ColumnType.INT64);
+        putType(ColumnType.UINT8);
+        putType(ColumnType.UINT16);
+        putType(ColumnType.UINT32);
+        putType(ColumnType.UINT64);
+        putType(ColumnType.FLOAT);
+        putType(ColumnType.DOUBLE);
+        putType(ColumnType.UUID);
+    }
+
+    private static void putType(ColumnType type) {
+        types.put(type.typeSpec().name(), type);
+    }
+
+    /**
+     * Convert SortedIndexColumn to IndexColumnChange.
+     * @param col IndexColumnChange.
+     * @param colInit IndexColumnChange to fullfill.
+     */
+    public static void convert(SortedIndexColumn col, IndexColumnChange colInit) {
+        colInit.changeName(col.name());
+        colInit.changeAsc(col.asc());
+    }
+
+    /**
+     * Convert IndexColumnView to SortedIndexColumn.
+     *
+     * @param colCfg IndexColumnView.
+     * @return SortedIndexColumn.
+     */
+    public static SortedIndexColumn convert(IndexColumnView colCfg) {
+        return new SortedIndexColumnImpl(colCfg.name(), colCfg.asc());
+    }
+
+    /**
+     * Convert TableIndex to TableIndexChange.
+     *
+     * @param idx TableIndex.
+     * @param idxChg TableIndexChange to fullfill.
+     */
+    public static void convert(TableIndex idx, TableIndexChange idxChg) {
+        idxChg.changeName(idx.name());
+        idxChg.changeType(idx.type());
+        switch (idx.type()) {
+            case HASH_TYPE:
+                HashIndex hashIdx = (HashIndex)idx;
+
+                String[] colNames = hashIdx.columns().stream().map(IndexColumn::name).toArray(String[]::new);
+                idxChg.changeColNames(colNames);
+                break;
+            case PARTIAL_TYPE:
+                PartialIndex partIdx = (PartialIndex)idx;
+
+                idxChg.changeUniq(partIdx.unique());
+                idxChg.changeExpr(partIdx.expr());
+
+                idxChg.changeColumns(colsChg -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : partIdx.columns())
+                        colsChg.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+                break;
+            case SORTED_TYPE:
+                SortedIndex sortIdx = (SortedIndex)idx;
+                idxChg.changeUniq(sortIdx.unique());
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : sortIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+                break;

Review comment:
       ```suggestion
                   });
                   
                   break;
   ```

##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnChange;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexChange;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.schemas.table.TablesChange;
+import org.apache.ignite.configuration.tree.NamedListView;
+import org.apache.ignite.internal.schema.ColumnImpl;
+import org.apache.ignite.internal.schema.HashIndexImpl;
+import org.apache.ignite.internal.schema.PartialIndexImpl;
+import org.apache.ignite.internal.schema.PrimaryIndexImpl;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.SortedIndexColumnImpl;
+import org.apache.ignite.internal.schema.SortedIndexImpl;
+import org.apache.ignite.schema.Column;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.IndexColumn;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
+import org.apache.ignite.schema.SortedIndexColumn;
+import org.apache.ignite.schema.TableIndex;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+/**
+ * Configuration to schema and vice versa converter.
+ */
+public class SchemaConfigurationConverter {
+    /** Hash index type. */
+    private static final String HASH_TYPE = "HASH";
+
+    /** Sorted index type. */
+    private static final String SORTED_TYPE = "SORTED";
+
+    /** Partial index type. */
+    private static final String PARTIAL_TYPE = "PARTIAL";
+
+    /** Primary key index type. */
+    private static final String PK_TYPE = "PK";
+
+    /** Types map. */
+    private static Map<String, ColumnType> types = new HashMap<>();
+
+    static {
+        putType(ColumnType.INT8);
+        putType(ColumnType.INT16);
+        putType(ColumnType.INT32);
+        putType(ColumnType.INT64);
+        putType(ColumnType.UINT8);
+        putType(ColumnType.UINT16);
+        putType(ColumnType.UINT32);
+        putType(ColumnType.UINT64);
+        putType(ColumnType.FLOAT);
+        putType(ColumnType.DOUBLE);
+        putType(ColumnType.UUID);
+    }
+
+    private static void putType(ColumnType type) {
+        types.put(type.typeSpec().name(), type);
+    }
+
+    /**
+     * Convert SortedIndexColumn to IndexColumnChange.
+     * @param col IndexColumnChange.
+     * @param colInit IndexColumnChange to fullfill.
+     */
+    public static void convert(SortedIndexColumn col, IndexColumnChange colInit) {
+        colInit.changeName(col.name());
+        colInit.changeAsc(col.asc());
+    }
+
+    /**
+     * Convert IndexColumnView to SortedIndexColumn.
+     *
+     * @param colCfg IndexColumnView.
+     * @return SortedIndexColumn.
+     */
+    public static SortedIndexColumn convert(IndexColumnView colCfg) {
+        return new SortedIndexColumnImpl(colCfg.name(), colCfg.asc());
+    }
+
+    /**
+     * Convert TableIndex to TableIndexChange.
+     *
+     * @param idx TableIndex.
+     * @param idxChg TableIndexChange to fullfill.
+     */
+    public static void convert(TableIndex idx, TableIndexChange idxChg) {
+        idxChg.changeName(idx.name());
+        idxChg.changeType(idx.type());
+        switch (idx.type()) {
+            case HASH_TYPE:
+                HashIndex hashIdx = (HashIndex)idx;
+
+                String[] colNames = hashIdx.columns().stream().map(IndexColumn::name).toArray(String[]::new);
+                idxChg.changeColNames(colNames);
+                break;
+            case PARTIAL_TYPE:
+                PartialIndex partIdx = (PartialIndex)idx;
+
+                idxChg.changeUniq(partIdx.unique());
+                idxChg.changeExpr(partIdx.expr());
+
+                idxChg.changeColumns(colsChg -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : partIdx.columns())
+                        colsChg.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+                break;
+            case SORTED_TYPE:
+                SortedIndex sortIdx = (SortedIndex)idx;
+                idxChg.changeUniq(sortIdx.unique());
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : sortIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+                break;
+            case PK_TYPE:
+                PrimaryIndex primIdx = (PrimaryIndex)idx;
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : primIdx.columns())

Review comment:
       ```suggestion
                       int colIdx = 0;
                       
                       for (SortedIndexColumn col : primIdx.columns())
   ```

##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnChange;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexChange;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.schemas.table.TablesChange;
+import org.apache.ignite.configuration.tree.NamedListView;
+import org.apache.ignite.internal.schema.ColumnImpl;
+import org.apache.ignite.internal.schema.HashIndexImpl;
+import org.apache.ignite.internal.schema.PartialIndexImpl;
+import org.apache.ignite.internal.schema.PrimaryIndexImpl;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.SortedIndexColumnImpl;
+import org.apache.ignite.internal.schema.SortedIndexImpl;
+import org.apache.ignite.schema.Column;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.IndexColumn;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
+import org.apache.ignite.schema.SortedIndexColumn;
+import org.apache.ignite.schema.TableIndex;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+/**
+ * Configuration to schema and vice versa converter.
+ */
+public class SchemaConfigurationConverter {
+    /** Hash index type. */
+    private static final String HASH_TYPE = "HASH";
+
+    /** Sorted index type. */
+    private static final String SORTED_TYPE = "SORTED";
+
+    /** Partial index type. */
+    private static final String PARTIAL_TYPE = "PARTIAL";
+
+    /** Primary key index type. */
+    private static final String PK_TYPE = "PK";
+
+    /** Types map. */
+    private static Map<String, ColumnType> types = new HashMap<>();
+
+    static {
+        putType(ColumnType.INT8);
+        putType(ColumnType.INT16);
+        putType(ColumnType.INT32);
+        putType(ColumnType.INT64);
+        putType(ColumnType.UINT8);
+        putType(ColumnType.UINT16);
+        putType(ColumnType.UINT32);
+        putType(ColumnType.UINT64);
+        putType(ColumnType.FLOAT);
+        putType(ColumnType.DOUBLE);
+        putType(ColumnType.UUID);
+    }
+
+    private static void putType(ColumnType type) {
+        types.put(type.typeSpec().name(), type);
+    }
+
+    /**
+     * Convert SortedIndexColumn to IndexColumnChange.
+     * @param col IndexColumnChange.
+     * @param colInit IndexColumnChange to fullfill.
+     */
+    public static void convert(SortedIndexColumn col, IndexColumnChange colInit) {
+        colInit.changeName(col.name());
+        colInit.changeAsc(col.asc());
+    }
+
+    /**
+     * Convert IndexColumnView to SortedIndexColumn.
+     *
+     * @param colCfg IndexColumnView.
+     * @return SortedIndexColumn.
+     */
+    public static SortedIndexColumn convert(IndexColumnView colCfg) {
+        return new SortedIndexColumnImpl(colCfg.name(), colCfg.asc());
+    }
+
+    /**
+     * Convert TableIndex to TableIndexChange.
+     *
+     * @param idx TableIndex.
+     * @param idxChg TableIndexChange to fullfill.
+     */
+    public static void convert(TableIndex idx, TableIndexChange idxChg) {
+        idxChg.changeName(idx.name());
+        idxChg.changeType(idx.type());
+        switch (idx.type()) {
+            case HASH_TYPE:
+                HashIndex hashIdx = (HashIndex)idx;
+
+                String[] colNames = hashIdx.columns().stream().map(IndexColumn::name).toArray(String[]::new);
+                idxChg.changeColNames(colNames);
+                break;
+            case PARTIAL_TYPE:
+                PartialIndex partIdx = (PartialIndex)idx;
+
+                idxChg.changeUniq(partIdx.unique());
+                idxChg.changeExpr(partIdx.expr());
+
+                idxChg.changeColumns(colsChg -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : partIdx.columns())
+                        colsChg.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+                break;
+            case SORTED_TYPE:
+                SortedIndex sortIdx = (SortedIndex)idx;
+                idxChg.changeUniq(sortIdx.unique());
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : sortIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+                break;
+            case PK_TYPE:
+                PrimaryIndex primIdx = (PrimaryIndex)idx;
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : primIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+
+                idxChg.changeAffinityColumns(primIdx.affinityColumns().toArray(new String[primIdx.affinityColumns().size()]));
+                break;

Review comment:
       ```suggestion
                   idxChg.changeAffinityColumns(primIdx.affinityColumns().toArray(new String[primIdx.affinityColumns().size()]));
                   
                   break;
   ```

##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnChange;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexChange;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.schemas.table.TablesChange;
+import org.apache.ignite.configuration.tree.NamedListView;
+import org.apache.ignite.internal.schema.ColumnImpl;
+import org.apache.ignite.internal.schema.HashIndexImpl;
+import org.apache.ignite.internal.schema.PartialIndexImpl;
+import org.apache.ignite.internal.schema.PrimaryIndexImpl;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.SortedIndexColumnImpl;
+import org.apache.ignite.internal.schema.SortedIndexImpl;
+import org.apache.ignite.schema.Column;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.IndexColumn;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
+import org.apache.ignite.schema.SortedIndexColumn;
+import org.apache.ignite.schema.TableIndex;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+/**
+ * Configuration to schema and vice versa converter.
+ */
+public class SchemaConfigurationConverter {
+    /** Hash index type. */
+    private static final String HASH_TYPE = "HASH";
+
+    /** Sorted index type. */
+    private static final String SORTED_TYPE = "SORTED";
+
+    /** Partial index type. */
+    private static final String PARTIAL_TYPE = "PARTIAL";
+
+    /** Primary key index type. */
+    private static final String PK_TYPE = "PK";
+
+    /** Types map. */
+    private static Map<String, ColumnType> types = new HashMap<>();
+
+    static {
+        putType(ColumnType.INT8);
+        putType(ColumnType.INT16);
+        putType(ColumnType.INT32);
+        putType(ColumnType.INT64);
+        putType(ColumnType.UINT8);
+        putType(ColumnType.UINT16);
+        putType(ColumnType.UINT32);
+        putType(ColumnType.UINT64);
+        putType(ColumnType.FLOAT);
+        putType(ColumnType.DOUBLE);
+        putType(ColumnType.UUID);
+    }
+
+    private static void putType(ColumnType type) {
+        types.put(type.typeSpec().name(), type);
+    }
+
+    /**
+     * Convert SortedIndexColumn to IndexColumnChange.
+     * @param col IndexColumnChange.
+     * @param colInit IndexColumnChange to fullfill.
+     */
+    public static void convert(SortedIndexColumn col, IndexColumnChange colInit) {
+        colInit.changeName(col.name());
+        colInit.changeAsc(col.asc());
+    }
+
+    /**
+     * Convert IndexColumnView to SortedIndexColumn.
+     *
+     * @param colCfg IndexColumnView.
+     * @return SortedIndexColumn.
+     */
+    public static SortedIndexColumn convert(IndexColumnView colCfg) {
+        return new SortedIndexColumnImpl(colCfg.name(), colCfg.asc());
+    }
+
+    /**
+     * Convert TableIndex to TableIndexChange.
+     *
+     * @param idx TableIndex.
+     * @param idxChg TableIndexChange to fullfill.
+     */
+    public static void convert(TableIndex idx, TableIndexChange idxChg) {
+        idxChg.changeName(idx.name());
+        idxChg.changeType(idx.type());
+        switch (idx.type()) {
+            case HASH_TYPE:
+                HashIndex hashIdx = (HashIndex)idx;
+
+                String[] colNames = hashIdx.columns().stream().map(IndexColumn::name).toArray(String[]::new);
+                idxChg.changeColNames(colNames);
+                break;

Review comment:
       ```suggestion
                   String[] colNames = hashIdx.columns().stream().map(IndexColumn::name).toArray(String[]::new);
                   
                   idxChg.changeColNames(colNames);
                   
                   break;
   ```

##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnChange;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexChange;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.schemas.table.TablesChange;
+import org.apache.ignite.configuration.tree.NamedListView;
+import org.apache.ignite.internal.schema.ColumnImpl;
+import org.apache.ignite.internal.schema.HashIndexImpl;
+import org.apache.ignite.internal.schema.PartialIndexImpl;
+import org.apache.ignite.internal.schema.PrimaryIndexImpl;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.SortedIndexColumnImpl;
+import org.apache.ignite.internal.schema.SortedIndexImpl;
+import org.apache.ignite.schema.Column;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.IndexColumn;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
+import org.apache.ignite.schema.SortedIndexColumn;
+import org.apache.ignite.schema.TableIndex;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+/**
+ * Configuration to schema and vice versa converter.
+ */
+public class SchemaConfigurationConverter {
+    /** Hash index type. */
+    private static final String HASH_TYPE = "HASH";
+
+    /** Sorted index type. */
+    private static final String SORTED_TYPE = "SORTED";
+
+    /** Partial index type. */
+    private static final String PARTIAL_TYPE = "PARTIAL";
+
+    /** Primary key index type. */
+    private static final String PK_TYPE = "PK";
+
+    /** Types map. */
+    private static Map<String, ColumnType> types = new HashMap<>();
+
+    static {
+        putType(ColumnType.INT8);
+        putType(ColumnType.INT16);
+        putType(ColumnType.INT32);
+        putType(ColumnType.INT64);
+        putType(ColumnType.UINT8);
+        putType(ColumnType.UINT16);
+        putType(ColumnType.UINT32);
+        putType(ColumnType.UINT64);
+        putType(ColumnType.FLOAT);
+        putType(ColumnType.DOUBLE);
+        putType(ColumnType.UUID);
+    }
+
+    private static void putType(ColumnType type) {
+        types.put(type.typeSpec().name(), type);
+    }
+
+    /**
+     * Convert SortedIndexColumn to IndexColumnChange.
+     * @param col IndexColumnChange.
+     * @param colInit IndexColumnChange to fullfill.
+     */
+    public static void convert(SortedIndexColumn col, IndexColumnChange colInit) {
+        colInit.changeName(col.name());
+        colInit.changeAsc(col.asc());
+    }
+
+    /**
+     * Convert IndexColumnView to SortedIndexColumn.
+     *
+     * @param colCfg IndexColumnView.
+     * @return SortedIndexColumn.
+     */
+    public static SortedIndexColumn convert(IndexColumnView colCfg) {
+        return new SortedIndexColumnImpl(colCfg.name(), colCfg.asc());
+    }
+
+    /**
+     * Convert TableIndex to TableIndexChange.
+     *
+     * @param idx TableIndex.
+     * @param idxChg TableIndexChange to fullfill.
+     */
+    public static void convert(TableIndex idx, TableIndexChange idxChg) {
+        idxChg.changeName(idx.name());
+        idxChg.changeType(idx.type());
+        switch (idx.type()) {
+            case HASH_TYPE:
+                HashIndex hashIdx = (HashIndex)idx;
+
+                String[] colNames = hashIdx.columns().stream().map(IndexColumn::name).toArray(String[]::new);
+                idxChg.changeColNames(colNames);
+                break;
+            case PARTIAL_TYPE:
+                PartialIndex partIdx = (PartialIndex)idx;
+
+                idxChg.changeUniq(partIdx.unique());
+                idxChg.changeExpr(partIdx.expr());
+
+                idxChg.changeColumns(colsChg -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : partIdx.columns())
+                        colsChg.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+                break;
+            case SORTED_TYPE:
+                SortedIndex sortIdx = (SortedIndex)idx;
+                idxChg.changeUniq(sortIdx.unique());
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : sortIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+                break;
+            case PK_TYPE:
+                PrimaryIndex primIdx = (PrimaryIndex)idx;
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : primIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+
+                idxChg.changeAffinityColumns(primIdx.affinityColumns().toArray(new String[primIdx.affinityColumns().size()]));
+                break;
+            default:
+                throw new IllegalArgumentException("Unknown index type " + idx.type());
+        }
+    }
+
+    /**
+     * Convert TableIndexView into TableIndex.
+     *
+     * @param idxView TableIndexView.
+     * @return TableIndex.
+     */
+    public static TableIndex convert(TableIndexView idxView) {
+        String name = idxView.name();
+        String type = idxView.type();
+        switch (type) {

Review comment:
       ```suggestion
           String type = idxView.type();
           
           switch (type) {
   ```

##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnChange;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexChange;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.schemas.table.TablesChange;
+import org.apache.ignite.configuration.tree.NamedListView;
+import org.apache.ignite.internal.schema.ColumnImpl;
+import org.apache.ignite.internal.schema.HashIndexImpl;
+import org.apache.ignite.internal.schema.PartialIndexImpl;
+import org.apache.ignite.internal.schema.PrimaryIndexImpl;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.SortedIndexColumnImpl;
+import org.apache.ignite.internal.schema.SortedIndexImpl;
+import org.apache.ignite.schema.Column;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.IndexColumn;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
+import org.apache.ignite.schema.SortedIndexColumn;
+import org.apache.ignite.schema.TableIndex;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+/**
+ * Configuration to schema and vice versa converter.
+ */
+public class SchemaConfigurationConverter {
+    /** Hash index type. */
+    private static final String HASH_TYPE = "HASH";
+
+    /** Sorted index type. */
+    private static final String SORTED_TYPE = "SORTED";
+
+    /** Partial index type. */
+    private static final String PARTIAL_TYPE = "PARTIAL";
+
+    /** Primary key index type. */
+    private static final String PK_TYPE = "PK";
+
+    /** Types map. */
+    private static Map<String, ColumnType> types = new HashMap<>();
+
+    static {
+        putType(ColumnType.INT8);
+        putType(ColumnType.INT16);
+        putType(ColumnType.INT32);
+        putType(ColumnType.INT64);
+        putType(ColumnType.UINT8);
+        putType(ColumnType.UINT16);
+        putType(ColumnType.UINT32);
+        putType(ColumnType.UINT64);
+        putType(ColumnType.FLOAT);
+        putType(ColumnType.DOUBLE);
+        putType(ColumnType.UUID);
+    }
+
+    private static void putType(ColumnType type) {
+        types.put(type.typeSpec().name(), type);
+    }
+
+    /**
+     * Convert SortedIndexColumn to IndexColumnChange.
+     * @param col IndexColumnChange.
+     * @param colInit IndexColumnChange to fullfill.
+     */
+    public static void convert(SortedIndexColumn col, IndexColumnChange colInit) {
+        colInit.changeName(col.name());
+        colInit.changeAsc(col.asc());
+    }
+
+    /**
+     * Convert IndexColumnView to SortedIndexColumn.
+     *
+     * @param colCfg IndexColumnView.
+     * @return SortedIndexColumn.
+     */
+    public static SortedIndexColumn convert(IndexColumnView colCfg) {
+        return new SortedIndexColumnImpl(colCfg.name(), colCfg.asc());
+    }
+
+    /**
+     * Convert TableIndex to TableIndexChange.
+     *
+     * @param idx TableIndex.
+     * @param idxChg TableIndexChange to fullfill.
+     */
+    public static void convert(TableIndex idx, TableIndexChange idxChg) {
+        idxChg.changeName(idx.name());
+        idxChg.changeType(idx.type());
+        switch (idx.type()) {
+            case HASH_TYPE:
+                HashIndex hashIdx = (HashIndex)idx;
+
+                String[] colNames = hashIdx.columns().stream().map(IndexColumn::name).toArray(String[]::new);
+                idxChg.changeColNames(colNames);
+                break;
+            case PARTIAL_TYPE:
+                PartialIndex partIdx = (PartialIndex)idx;
+
+                idxChg.changeUniq(partIdx.unique());
+                idxChg.changeExpr(partIdx.expr());
+
+                idxChg.changeColumns(colsChg -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : partIdx.columns())
+                        colsChg.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+                break;
+            case SORTED_TYPE:
+                SortedIndex sortIdx = (SortedIndex)idx;
+                idxChg.changeUniq(sortIdx.unique());
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : sortIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+                break;
+            case PK_TYPE:
+                PrimaryIndex primIdx = (PrimaryIndex)idx;
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : primIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+
+                idxChg.changeAffinityColumns(primIdx.affinityColumns().toArray(new String[primIdx.affinityColumns().size()]));
+                break;
+            default:
+                throw new IllegalArgumentException("Unknown index type " + idx.type());
+        }
+    }
+
+    /**
+     * Convert TableIndexView into TableIndex.
+     *
+     * @param idxView TableIndexView.
+     * @return TableIndex.
+     */
+    public static TableIndex convert(TableIndexView idxView) {
+        String name = idxView.name();
+        String type = idxView.type();
+        switch (type) {
+            case "HASH":
+                String[] hashCols = idxView.colNames();
+
+                return new HashIndexImpl(name, hashCols);
+
+            case "SORTED":
+                boolean sortedUniq = idxView.uniq();
+                SortedMap<Integer, SortedIndexColumn> sortedCols = new TreeMap();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    sortedCols.put(Integer.valueOf(key), col);
+                }
+
+                return new SortedIndexImpl(name, new ArrayList<>(sortedCols.values()), sortedUniq);
+
+            case "PARTIAL":
+                boolean partialUniq = idxView.uniq();
+                String expr = idxView.expr();
+                NamedListView<? extends IndexColumnView> colsView = idxView.columns();
+                SortedMap<Integer, SortedIndexColumn> partialCols = new TreeMap<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(colsView.get(key));
+                    partialCols.put(Integer.valueOf(key), col);
+                }
+
+                return new PartialIndexImpl(name, new ArrayList<>(partialCols.values()), partialUniq, expr);
+
+            case "PK":
+                SortedMap<Integer, SortedIndexColumn> cols = new TreeMap<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    cols.put(Integer.valueOf(key), col);
+                }
+                String[] affCols = idxView.affinityColumns();
+
+                return new PrimaryIndexImpl(new ArrayList(cols.values()), List.of(affCols));
+
+            default:
+                throw new IllegalArgumentException("Unknown type " + type);
+        }
+    }
+
+    /**
+     * Convert ColumnType to ColumnTypeChange.
+     *
+     * @param colType ColumnType.
+     * @param colTypeChg ColumnTypeChange to fullfill.
+     */
+    public static void convert(ColumnType colType, ColumnTypeChange colTypeChg) {
+        String typeName = colType.typeSpec().name();
+        if (types.containsKey(typeName))

Review comment:
       ```suggestion
           String typeName = colType.typeSpec().name();
           
           if (types.containsKey(typeName))
   ```

##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnChange;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexChange;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.schemas.table.TablesChange;
+import org.apache.ignite.configuration.tree.NamedListView;
+import org.apache.ignite.internal.schema.ColumnImpl;
+import org.apache.ignite.internal.schema.HashIndexImpl;
+import org.apache.ignite.internal.schema.PartialIndexImpl;
+import org.apache.ignite.internal.schema.PrimaryIndexImpl;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.SortedIndexColumnImpl;
+import org.apache.ignite.internal.schema.SortedIndexImpl;
+import org.apache.ignite.schema.Column;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.IndexColumn;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
+import org.apache.ignite.schema.SortedIndexColumn;
+import org.apache.ignite.schema.TableIndex;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+/**
+ * Configuration to schema and vice versa converter.
+ */
+public class SchemaConfigurationConverter {
+    /** Hash index type. */
+    private static final String HASH_TYPE = "HASH";
+
+    /** Sorted index type. */
+    private static final String SORTED_TYPE = "SORTED";
+
+    /** Partial index type. */
+    private static final String PARTIAL_TYPE = "PARTIAL";
+
+    /** Primary key index type. */
+    private static final String PK_TYPE = "PK";
+
+    /** Types map. */
+    private static Map<String, ColumnType> types = new HashMap<>();
+
+    static {
+        putType(ColumnType.INT8);
+        putType(ColumnType.INT16);
+        putType(ColumnType.INT32);
+        putType(ColumnType.INT64);
+        putType(ColumnType.UINT8);
+        putType(ColumnType.UINT16);
+        putType(ColumnType.UINT32);
+        putType(ColumnType.UINT64);
+        putType(ColumnType.FLOAT);
+        putType(ColumnType.DOUBLE);
+        putType(ColumnType.UUID);
+    }
+
+    private static void putType(ColumnType type) {
+        types.put(type.typeSpec().name(), type);
+    }
+
+    /**
+     * Convert SortedIndexColumn to IndexColumnChange.
+     * @param col IndexColumnChange.
+     * @param colInit IndexColumnChange to fullfill.
+     */
+    public static void convert(SortedIndexColumn col, IndexColumnChange colInit) {
+        colInit.changeName(col.name());
+        colInit.changeAsc(col.asc());
+    }
+
+    /**
+     * Convert IndexColumnView to SortedIndexColumn.
+     *
+     * @param colCfg IndexColumnView.
+     * @return SortedIndexColumn.
+     */
+    public static SortedIndexColumn convert(IndexColumnView colCfg) {
+        return new SortedIndexColumnImpl(colCfg.name(), colCfg.asc());
+    }
+
+    /**
+     * Convert TableIndex to TableIndexChange.
+     *
+     * @param idx TableIndex.
+     * @param idxChg TableIndexChange to fullfill.
+     */
+    public static void convert(TableIndex idx, TableIndexChange idxChg) {
+        idxChg.changeName(idx.name());
+        idxChg.changeType(idx.type());
+        switch (idx.type()) {
+            case HASH_TYPE:
+                HashIndex hashIdx = (HashIndex)idx;
+
+                String[] colNames = hashIdx.columns().stream().map(IndexColumn::name).toArray(String[]::new);
+                idxChg.changeColNames(colNames);
+                break;
+            case PARTIAL_TYPE:
+                PartialIndex partIdx = (PartialIndex)idx;
+
+                idxChg.changeUniq(partIdx.unique());
+                idxChg.changeExpr(partIdx.expr());
+
+                idxChg.changeColumns(colsChg -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : partIdx.columns())
+                        colsChg.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+                break;
+            case SORTED_TYPE:
+                SortedIndex sortIdx = (SortedIndex)idx;
+                idxChg.changeUniq(sortIdx.unique());
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : sortIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+                break;
+            case PK_TYPE:
+                PrimaryIndex primIdx = (PrimaryIndex)idx;
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : primIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+
+                idxChg.changeAffinityColumns(primIdx.affinityColumns().toArray(new String[primIdx.affinityColumns().size()]));
+                break;
+            default:
+                throw new IllegalArgumentException("Unknown index type " + idx.type());
+        }
+    }
+
+    /**
+     * Convert TableIndexView into TableIndex.
+     *
+     * @param idxView TableIndexView.
+     * @return TableIndex.
+     */
+    public static TableIndex convert(TableIndexView idxView) {
+        String name = idxView.name();
+        String type = idxView.type();
+        switch (type) {
+            case "HASH":
+                String[] hashCols = idxView.colNames();
+
+                return new HashIndexImpl(name, hashCols);
+
+            case "SORTED":
+                boolean sortedUniq = idxView.uniq();
+                SortedMap<Integer, SortedIndexColumn> sortedCols = new TreeMap();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    sortedCols.put(Integer.valueOf(key), col);
+                }
+
+                return new SortedIndexImpl(name, new ArrayList<>(sortedCols.values()), sortedUniq);
+
+            case "PARTIAL":
+                boolean partialUniq = idxView.uniq();
+                String expr = idxView.expr();
+                NamedListView<? extends IndexColumnView> colsView = idxView.columns();
+                SortedMap<Integer, SortedIndexColumn> partialCols = new TreeMap<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(colsView.get(key));
+                    partialCols.put(Integer.valueOf(key), col);
+                }
+
+                return new PartialIndexImpl(name, new ArrayList<>(partialCols.values()), partialUniq, expr);

Review comment:
       ```suggestion
                   NamedListView<? extends IndexColumnView> colsView = idxView.columns();
                   
                   SortedMap<Integer, SortedIndexColumn> partialCols = new TreeMap<>();
                   
                   for (String key : idxView.columns().namedListKeys()) {
                       SortedIndexColumn col = convert(colsView.get(key));
                       
                       partialCols.put(Integer.valueOf(key), col);
                   }
   
                   return new PartialIndexImpl(name, new ArrayList<>(partialCols.values()), idxView.uniq(), idxView.expr());
   ```

##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnChange;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexChange;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.schemas.table.TablesChange;
+import org.apache.ignite.configuration.tree.NamedListView;
+import org.apache.ignite.internal.schema.ColumnImpl;
+import org.apache.ignite.internal.schema.HashIndexImpl;
+import org.apache.ignite.internal.schema.PartialIndexImpl;
+import org.apache.ignite.internal.schema.PrimaryIndexImpl;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.SortedIndexColumnImpl;
+import org.apache.ignite.internal.schema.SortedIndexImpl;
+import org.apache.ignite.schema.Column;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.IndexColumn;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
+import org.apache.ignite.schema.SortedIndexColumn;
+import org.apache.ignite.schema.TableIndex;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+/**
+ * Configuration to schema and vice versa converter.
+ */
+public class SchemaConfigurationConverter {
+    /** Hash index type. */
+    private static final String HASH_TYPE = "HASH";
+
+    /** Sorted index type. */
+    private static final String SORTED_TYPE = "SORTED";
+
+    /** Partial index type. */
+    private static final String PARTIAL_TYPE = "PARTIAL";
+
+    /** Primary key index type. */
+    private static final String PK_TYPE = "PK";
+
+    /** Types map. */
+    private static Map<String, ColumnType> types = new HashMap<>();
+
+    static {
+        putType(ColumnType.INT8);
+        putType(ColumnType.INT16);
+        putType(ColumnType.INT32);
+        putType(ColumnType.INT64);
+        putType(ColumnType.UINT8);
+        putType(ColumnType.UINT16);
+        putType(ColumnType.UINT32);
+        putType(ColumnType.UINT64);
+        putType(ColumnType.FLOAT);
+        putType(ColumnType.DOUBLE);
+        putType(ColumnType.UUID);
+    }
+
+    private static void putType(ColumnType type) {
+        types.put(type.typeSpec().name(), type);
+    }
+
+    /**
+     * Convert SortedIndexColumn to IndexColumnChange.
+     * @param col IndexColumnChange.
+     * @param colInit IndexColumnChange to fullfill.
+     */
+    public static void convert(SortedIndexColumn col, IndexColumnChange colInit) {
+        colInit.changeName(col.name());
+        colInit.changeAsc(col.asc());
+    }
+
+    /**
+     * Convert IndexColumnView to SortedIndexColumn.
+     *
+     * @param colCfg IndexColumnView.
+     * @return SortedIndexColumn.
+     */
+    public static SortedIndexColumn convert(IndexColumnView colCfg) {
+        return new SortedIndexColumnImpl(colCfg.name(), colCfg.asc());
+    }
+
+    /**
+     * Convert TableIndex to TableIndexChange.
+     *
+     * @param idx TableIndex.
+     * @param idxChg TableIndexChange to fullfill.
+     */
+    public static void convert(TableIndex idx, TableIndexChange idxChg) {
+        idxChg.changeName(idx.name());
+        idxChg.changeType(idx.type());
+        switch (idx.type()) {
+            case HASH_TYPE:
+                HashIndex hashIdx = (HashIndex)idx;
+
+                String[] colNames = hashIdx.columns().stream().map(IndexColumn::name).toArray(String[]::new);
+                idxChg.changeColNames(colNames);
+                break;
+            case PARTIAL_TYPE:
+                PartialIndex partIdx = (PartialIndex)idx;
+
+                idxChg.changeUniq(partIdx.unique());
+                idxChg.changeExpr(partIdx.expr());
+
+                idxChg.changeColumns(colsChg -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : partIdx.columns())
+                        colsChg.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+                break;
+            case SORTED_TYPE:
+                SortedIndex sortIdx = (SortedIndex)idx;
+                idxChg.changeUniq(sortIdx.unique());
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : sortIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+                break;
+            case PK_TYPE:
+                PrimaryIndex primIdx = (PrimaryIndex)idx;
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : primIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+
+                idxChg.changeAffinityColumns(primIdx.affinityColumns().toArray(new String[primIdx.affinityColumns().size()]));
+                break;
+            default:
+                throw new IllegalArgumentException("Unknown index type " + idx.type());
+        }
+    }
+
+    /**
+     * Convert TableIndexView into TableIndex.
+     *
+     * @param idxView TableIndexView.
+     * @return TableIndex.
+     */
+    public static TableIndex convert(TableIndexView idxView) {
+        String name = idxView.name();
+        String type = idxView.type();
+        switch (type) {
+            case "HASH":
+                String[] hashCols = idxView.colNames();
+
+                return new HashIndexImpl(name, hashCols);
+
+            case "SORTED":
+                boolean sortedUniq = idxView.uniq();
+                SortedMap<Integer, SortedIndexColumn> sortedCols = new TreeMap();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    sortedCols.put(Integer.valueOf(key), col);
+                }
+
+                return new SortedIndexImpl(name, new ArrayList<>(sortedCols.values()), sortedUniq);
+
+            case "PARTIAL":
+                boolean partialUniq = idxView.uniq();
+                String expr = idxView.expr();
+                NamedListView<? extends IndexColumnView> colsView = idxView.columns();
+                SortedMap<Integer, SortedIndexColumn> partialCols = new TreeMap<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(colsView.get(key));
+                    partialCols.put(Integer.valueOf(key), col);
+                }
+
+                return new PartialIndexImpl(name, new ArrayList<>(partialCols.values()), partialUniq, expr);
+
+            case "PK":
+                SortedMap<Integer, SortedIndexColumn> cols = new TreeMap<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    cols.put(Integer.valueOf(key), col);
+                }
+                String[] affCols = idxView.affinityColumns();

Review comment:
       ```suggestion
                   SortedMap<Integer, SortedIndexColumn> cols = new TreeMap<>();
                   
                   for (String key : idxView.columns().namedListKeys()) {
                       SortedIndexColumn col = convert(idxView.columns().get(key));
                       
                       cols.put(Integer.valueOf(key), col);
                   }
                   
                   String[] affCols = idxView.affinityColumns();
   ```

##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnChange;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexChange;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.schemas.table.TablesChange;
+import org.apache.ignite.configuration.tree.NamedListView;
+import org.apache.ignite.internal.schema.ColumnImpl;
+import org.apache.ignite.internal.schema.HashIndexImpl;
+import org.apache.ignite.internal.schema.PartialIndexImpl;
+import org.apache.ignite.internal.schema.PrimaryIndexImpl;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.SortedIndexColumnImpl;
+import org.apache.ignite.internal.schema.SortedIndexImpl;
+import org.apache.ignite.schema.Column;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.IndexColumn;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
+import org.apache.ignite.schema.SortedIndexColumn;
+import org.apache.ignite.schema.TableIndex;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+/**
+ * Configuration to schema and vice versa converter.
+ */
+public class SchemaConfigurationConverter {
+    /** Hash index type. */
+    private static final String HASH_TYPE = "HASH";
+
+    /** Sorted index type. */
+    private static final String SORTED_TYPE = "SORTED";
+
+    /** Partial index type. */
+    private static final String PARTIAL_TYPE = "PARTIAL";
+
+    /** Primary key index type. */
+    private static final String PK_TYPE = "PK";
+
+    /** Types map. */
+    private static Map<String, ColumnType> types = new HashMap<>();
+
+    static {
+        putType(ColumnType.INT8);
+        putType(ColumnType.INT16);
+        putType(ColumnType.INT32);
+        putType(ColumnType.INT64);
+        putType(ColumnType.UINT8);
+        putType(ColumnType.UINT16);
+        putType(ColumnType.UINT32);
+        putType(ColumnType.UINT64);
+        putType(ColumnType.FLOAT);
+        putType(ColumnType.DOUBLE);
+        putType(ColumnType.UUID);
+    }
+
+    private static void putType(ColumnType type) {
+        types.put(type.typeSpec().name(), type);
+    }
+
+    /**
+     * Convert SortedIndexColumn to IndexColumnChange.
+     * @param col IndexColumnChange.
+     * @param colInit IndexColumnChange to fullfill.
+     */
+    public static void convert(SortedIndexColumn col, IndexColumnChange colInit) {
+        colInit.changeName(col.name());
+        colInit.changeAsc(col.asc());
+    }
+
+    /**
+     * Convert IndexColumnView to SortedIndexColumn.
+     *
+     * @param colCfg IndexColumnView.
+     * @return SortedIndexColumn.
+     */
+    public static SortedIndexColumn convert(IndexColumnView colCfg) {
+        return new SortedIndexColumnImpl(colCfg.name(), colCfg.asc());
+    }
+
+    /**
+     * Convert TableIndex to TableIndexChange.
+     *
+     * @param idx TableIndex.
+     * @param idxChg TableIndexChange to fullfill.
+     */
+    public static void convert(TableIndex idx, TableIndexChange idxChg) {
+        idxChg.changeName(idx.name());
+        idxChg.changeType(idx.type());
+        switch (idx.type()) {
+            case HASH_TYPE:
+                HashIndex hashIdx = (HashIndex)idx;
+
+                String[] colNames = hashIdx.columns().stream().map(IndexColumn::name).toArray(String[]::new);
+                idxChg.changeColNames(colNames);
+                break;
+            case PARTIAL_TYPE:
+                PartialIndex partIdx = (PartialIndex)idx;
+
+                idxChg.changeUniq(partIdx.unique());
+                idxChg.changeExpr(partIdx.expr());
+
+                idxChg.changeColumns(colsChg -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : partIdx.columns())
+                        colsChg.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+                break;
+            case SORTED_TYPE:
+                SortedIndex sortIdx = (SortedIndex)idx;
+                idxChg.changeUniq(sortIdx.unique());
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : sortIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+                break;
+            case PK_TYPE:
+                PrimaryIndex primIdx = (PrimaryIndex)idx;
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : primIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+
+                idxChg.changeAffinityColumns(primIdx.affinityColumns().toArray(new String[primIdx.affinityColumns().size()]));
+                break;
+            default:
+                throw new IllegalArgumentException("Unknown index type " + idx.type());
+        }
+    }
+
+    /**
+     * Convert TableIndexView into TableIndex.
+     *
+     * @param idxView TableIndexView.
+     * @return TableIndex.
+     */
+    public static TableIndex convert(TableIndexView idxView) {
+        String name = idxView.name();
+        String type = idxView.type();
+        switch (type) {
+            case "HASH":
+                String[] hashCols = idxView.colNames();
+
+                return new HashIndexImpl(name, hashCols);
+
+            case "SORTED":
+                boolean sortedUniq = idxView.uniq();
+                SortedMap<Integer, SortedIndexColumn> sortedCols = new TreeMap();
+                for (String key : idxView.columns().namedListKeys()) {

Review comment:
       ```suggestion
                   boolean sortedUniq = idxView.uniq();
                   SortedMap<Integer, SortedIndexColumn> sortedCols = new TreeMap();
                   
                   for (String key : idxView.columns().namedListKeys()) {
   ```

##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnChange;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexChange;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.schemas.table.TablesChange;
+import org.apache.ignite.configuration.tree.NamedListView;
+import org.apache.ignite.internal.schema.ColumnImpl;
+import org.apache.ignite.internal.schema.HashIndexImpl;
+import org.apache.ignite.internal.schema.PartialIndexImpl;
+import org.apache.ignite.internal.schema.PrimaryIndexImpl;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.SortedIndexColumnImpl;
+import org.apache.ignite.internal.schema.SortedIndexImpl;
+import org.apache.ignite.schema.Column;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.IndexColumn;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
+import org.apache.ignite.schema.SortedIndexColumn;
+import org.apache.ignite.schema.TableIndex;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+/**
+ * Configuration to schema and vice versa converter.
+ */
+public class SchemaConfigurationConverter {
+    /** Hash index type. */
+    private static final String HASH_TYPE = "HASH";
+
+    /** Sorted index type. */
+    private static final String SORTED_TYPE = "SORTED";
+
+    /** Partial index type. */
+    private static final String PARTIAL_TYPE = "PARTIAL";
+
+    /** Primary key index type. */
+    private static final String PK_TYPE = "PK";
+
+    /** Types map. */
+    private static Map<String, ColumnType> types = new HashMap<>();
+
+    static {
+        putType(ColumnType.INT8);
+        putType(ColumnType.INT16);
+        putType(ColumnType.INT32);
+        putType(ColumnType.INT64);
+        putType(ColumnType.UINT8);
+        putType(ColumnType.UINT16);
+        putType(ColumnType.UINT32);
+        putType(ColumnType.UINT64);
+        putType(ColumnType.FLOAT);
+        putType(ColumnType.DOUBLE);
+        putType(ColumnType.UUID);
+    }
+
+    private static void putType(ColumnType type) {
+        types.put(type.typeSpec().name(), type);
+    }
+
+    /**
+     * Convert SortedIndexColumn to IndexColumnChange.
+     * @param col IndexColumnChange.
+     * @param colInit IndexColumnChange to fullfill.
+     */
+    public static void convert(SortedIndexColumn col, IndexColumnChange colInit) {
+        colInit.changeName(col.name());
+        colInit.changeAsc(col.asc());
+    }
+
+    /**
+     * Convert IndexColumnView to SortedIndexColumn.
+     *
+     * @param colCfg IndexColumnView.
+     * @return SortedIndexColumn.
+     */
+    public static SortedIndexColumn convert(IndexColumnView colCfg) {
+        return new SortedIndexColumnImpl(colCfg.name(), colCfg.asc());
+    }
+
+    /**
+     * Convert TableIndex to TableIndexChange.
+     *
+     * @param idx TableIndex.
+     * @param idxChg TableIndexChange to fullfill.
+     */
+    public static void convert(TableIndex idx, TableIndexChange idxChg) {
+        idxChg.changeName(idx.name());
+        idxChg.changeType(idx.type());
+        switch (idx.type()) {
+            case HASH_TYPE:
+                HashIndex hashIdx = (HashIndex)idx;
+
+                String[] colNames = hashIdx.columns().stream().map(IndexColumn::name).toArray(String[]::new);
+                idxChg.changeColNames(colNames);
+                break;
+            case PARTIAL_TYPE:
+                PartialIndex partIdx = (PartialIndex)idx;
+
+                idxChg.changeUniq(partIdx.unique());
+                idxChg.changeExpr(partIdx.expr());
+
+                idxChg.changeColumns(colsChg -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : partIdx.columns())
+                        colsChg.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+                break;
+            case SORTED_TYPE:
+                SortedIndex sortIdx = (SortedIndex)idx;
+                idxChg.changeUniq(sortIdx.unique());
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : sortIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+                break;
+            case PK_TYPE:
+                PrimaryIndex primIdx = (PrimaryIndex)idx;
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : primIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+
+                idxChg.changeAffinityColumns(primIdx.affinityColumns().toArray(new String[primIdx.affinityColumns().size()]));
+                break;
+            default:
+                throw new IllegalArgumentException("Unknown index type " + idx.type());
+        }
+    }
+
+    /**
+     * Convert TableIndexView into TableIndex.
+     *
+     * @param idxView TableIndexView.
+     * @return TableIndex.
+     */
+    public static TableIndex convert(TableIndexView idxView) {
+        String name = idxView.name();
+        String type = idxView.type();
+        switch (type) {
+            case "HASH":
+                String[] hashCols = idxView.colNames();
+
+                return new HashIndexImpl(name, hashCols);
+
+            case "SORTED":
+                boolean sortedUniq = idxView.uniq();
+                SortedMap<Integer, SortedIndexColumn> sortedCols = new TreeMap();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    sortedCols.put(Integer.valueOf(key), col);
+                }
+
+                return new SortedIndexImpl(name, new ArrayList<>(sortedCols.values()), sortedUniq);
+
+            case "PARTIAL":
+                boolean partialUniq = idxView.uniq();
+                String expr = idxView.expr();
+                NamedListView<? extends IndexColumnView> colsView = idxView.columns();
+                SortedMap<Integer, SortedIndexColumn> partialCols = new TreeMap<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(colsView.get(key));
+                    partialCols.put(Integer.valueOf(key), col);
+                }
+
+                return new PartialIndexImpl(name, new ArrayList<>(partialCols.values()), partialUniq, expr);
+
+            case "PK":
+                SortedMap<Integer, SortedIndexColumn> cols = new TreeMap<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    cols.put(Integer.valueOf(key), col);
+                }
+                String[] affCols = idxView.affinityColumns();
+
+                return new PrimaryIndexImpl(new ArrayList(cols.values()), List.of(affCols));
+
+            default:
+                throw new IllegalArgumentException("Unknown type " + type);
+        }
+    }
+
+    /**
+     * Convert ColumnType to ColumnTypeChange.
+     *
+     * @param colType ColumnType.
+     * @param colTypeChg ColumnTypeChange to fullfill.
+     */
+    public static void convert(ColumnType colType, ColumnTypeChange colTypeChg) {
+        String typeName = colType.typeSpec().name();
+        if (types.containsKey(typeName))
+            colTypeChg.changeType(typeName);
+        else {
+            colTypeChg.changeType(typeName);
+            switch (typeName) {
+                case "BITMASK":
+                case "BLOB":
+                case "STRING":
+                    ColumnType.VarLenColumnType varLenColType = (ColumnType.VarLenColumnType)colType;
+
+                    colTypeChg.changeLength(varLenColType.length());
+                    break;
+
+                case "DECIMAL":
+                    ColumnType.NumericColumnType numColType = (ColumnType.NumericColumnType)colType;
+
+                    colTypeChg.changePrecision(numColType.precision());
+                    colTypeChg.changeScale(numColType.scale());
+                    break;

Review comment:
       ```suggestion
                       colTypeChg.changeScale(numColType.scale());
                       
                       break;
   ```

##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnChange;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexChange;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.schemas.table.TablesChange;
+import org.apache.ignite.configuration.tree.NamedListView;
+import org.apache.ignite.internal.schema.ColumnImpl;
+import org.apache.ignite.internal.schema.HashIndexImpl;
+import org.apache.ignite.internal.schema.PartialIndexImpl;
+import org.apache.ignite.internal.schema.PrimaryIndexImpl;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.SortedIndexColumnImpl;
+import org.apache.ignite.internal.schema.SortedIndexImpl;
+import org.apache.ignite.schema.Column;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.IndexColumn;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
+import org.apache.ignite.schema.SortedIndexColumn;
+import org.apache.ignite.schema.TableIndex;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+/**
+ * Configuration to schema and vice versa converter.
+ */
+public class SchemaConfigurationConverter {
+    /** Hash index type. */
+    private static final String HASH_TYPE = "HASH";
+
+    /** Sorted index type. */
+    private static final String SORTED_TYPE = "SORTED";
+
+    /** Partial index type. */
+    private static final String PARTIAL_TYPE = "PARTIAL";
+
+    /** Primary key index type. */
+    private static final String PK_TYPE = "PK";
+
+    /** Types map. */
+    private static Map<String, ColumnType> types = new HashMap<>();
+
+    static {
+        putType(ColumnType.INT8);
+        putType(ColumnType.INT16);
+        putType(ColumnType.INT32);
+        putType(ColumnType.INT64);
+        putType(ColumnType.UINT8);
+        putType(ColumnType.UINT16);
+        putType(ColumnType.UINT32);
+        putType(ColumnType.UINT64);
+        putType(ColumnType.FLOAT);
+        putType(ColumnType.DOUBLE);
+        putType(ColumnType.UUID);
+    }
+
+    private static void putType(ColumnType type) {
+        types.put(type.typeSpec().name(), type);
+    }
+
+    /**
+     * Convert SortedIndexColumn to IndexColumnChange.
+     * @param col IndexColumnChange.
+     * @param colInit IndexColumnChange to fullfill.
+     */
+    public static void convert(SortedIndexColumn col, IndexColumnChange colInit) {
+        colInit.changeName(col.name());
+        colInit.changeAsc(col.asc());
+    }
+
+    /**
+     * Convert IndexColumnView to SortedIndexColumn.
+     *
+     * @param colCfg IndexColumnView.
+     * @return SortedIndexColumn.
+     */
+    public static SortedIndexColumn convert(IndexColumnView colCfg) {
+        return new SortedIndexColumnImpl(colCfg.name(), colCfg.asc());
+    }
+
+    /**
+     * Convert TableIndex to TableIndexChange.
+     *
+     * @param idx TableIndex.
+     * @param idxChg TableIndexChange to fullfill.
+     */
+    public static void convert(TableIndex idx, TableIndexChange idxChg) {
+        idxChg.changeName(idx.name());
+        idxChg.changeType(idx.type());
+        switch (idx.type()) {
+            case HASH_TYPE:
+                HashIndex hashIdx = (HashIndex)idx;
+
+                String[] colNames = hashIdx.columns().stream().map(IndexColumn::name).toArray(String[]::new);
+                idxChg.changeColNames(colNames);
+                break;
+            case PARTIAL_TYPE:
+                PartialIndex partIdx = (PartialIndex)idx;
+
+                idxChg.changeUniq(partIdx.unique());
+                idxChg.changeExpr(partIdx.expr());
+
+                idxChg.changeColumns(colsChg -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : partIdx.columns())
+                        colsChg.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+                break;
+            case SORTED_TYPE:
+                SortedIndex sortIdx = (SortedIndex)idx;
+                idxChg.changeUniq(sortIdx.unique());
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : sortIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+                break;
+            case PK_TYPE:
+                PrimaryIndex primIdx = (PrimaryIndex)idx;
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : primIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+
+                idxChg.changeAffinityColumns(primIdx.affinityColumns().toArray(new String[primIdx.affinityColumns().size()]));
+                break;
+            default:
+                throw new IllegalArgumentException("Unknown index type " + idx.type());
+        }
+    }
+
+    /**
+     * Convert TableIndexView into TableIndex.
+     *
+     * @param idxView TableIndexView.
+     * @return TableIndex.
+     */
+    public static TableIndex convert(TableIndexView idxView) {
+        String name = idxView.name();
+        String type = idxView.type();
+        switch (type) {
+            case "HASH":
+                String[] hashCols = idxView.colNames();
+
+                return new HashIndexImpl(name, hashCols);
+
+            case "SORTED":
+                boolean sortedUniq = idxView.uniq();
+                SortedMap<Integer, SortedIndexColumn> sortedCols = new TreeMap();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    sortedCols.put(Integer.valueOf(key), col);
+                }
+
+                return new SortedIndexImpl(name, new ArrayList<>(sortedCols.values()), sortedUniq);
+
+            case "PARTIAL":
+                boolean partialUniq = idxView.uniq();
+                String expr = idxView.expr();
+                NamedListView<? extends IndexColumnView> colsView = idxView.columns();
+                SortedMap<Integer, SortedIndexColumn> partialCols = new TreeMap<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(colsView.get(key));
+                    partialCols.put(Integer.valueOf(key), col);
+                }
+
+                return new PartialIndexImpl(name, new ArrayList<>(partialCols.values()), partialUniq, expr);
+
+            case "PK":
+                SortedMap<Integer, SortedIndexColumn> cols = new TreeMap<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    cols.put(Integer.valueOf(key), col);
+                }
+                String[] affCols = idxView.affinityColumns();
+
+                return new PrimaryIndexImpl(new ArrayList(cols.values()), List.of(affCols));
+
+            default:
+                throw new IllegalArgumentException("Unknown type " + type);
+        }
+    }
+
+    /**
+     * Convert ColumnType to ColumnTypeChange.
+     *
+     * @param colType ColumnType.
+     * @param colTypeChg ColumnTypeChange to fullfill.
+     */
+    public static void convert(ColumnType colType, ColumnTypeChange colTypeChg) {
+        String typeName = colType.typeSpec().name();
+        if (types.containsKey(typeName))
+            colTypeChg.changeType(typeName);
+        else {
+            colTypeChg.changeType(typeName);
+            switch (typeName) {
+                case "BITMASK":
+                case "BLOB":
+                case "STRING":
+                    ColumnType.VarLenColumnType varLenColType = (ColumnType.VarLenColumnType)colType;
+
+                    colTypeChg.changeLength(varLenColType.length());
+                    break;
+
+                case "DECIMAL":
+                    ColumnType.NumericColumnType numColType = (ColumnType.NumericColumnType)colType;
+
+                    colTypeChg.changePrecision(numColType.precision());
+                    colTypeChg.changeScale(numColType.scale());
+                    break;
+                default:
+                    throw new IllegalArgumentException("Unknown type " + colType.typeSpec().name());
+            }
+        }
+    }
+
+    /**
+     * Convert ColumnTypeView to ColumnType.
+     *
+     * @param colTypeView ColumnTypeView.
+     * @return ColumnType.
+     */
+    public static ColumnType convert(ColumnTypeView colTypeView) {
+        String typeName = colTypeView.type();
+        ColumnType res = types.get(typeName);
+        if (res != null)
+            return res;
+        else {
+            switch (typeName) {
+                case "BITMASK":
+                    int bitmaskLen = colTypeView.length();
+
+                    return ColumnType.bitmaskOf(bitmaskLen);
+
+                case "STRING":
+                    int strLen = colTypeView.length();
+
+                    return ColumnType.stringOf(strLen);
+
+                case "BLOB":
+                    int blobLen = colTypeView.length();
+
+                    return ColumnType.blobOf(blobLen);
+
+                case "DECIMAL":
+                    int prec = colTypeView.precision();
+                    int scale = colTypeView.scale();
+
+                    return ColumnType.number(prec, scale);
+
+                default:
+                    throw new IllegalArgumentException("Unknown type " + typeName);
+            }
+        }
+    }
+
+    /**
+     * Convert column to column change.
+     *
+     * @param col Column to convert.
+     * @param colChg Column
+     */
+    public static void convert(Column col, ColumnChange colChg) {
+        colChg.changeName(col.name());
+        colChg.changeType(colTypeInit -> convert(col.type(), colTypeInit));
+        if (col.defaultValue() != null)
+            colChg.changeDefaultValue(col.defaultValue().toString());
+        colChg.changeNullable(col.nullable());

Review comment:
       ```suggestion
           colChg.changeName(col.name());
           colChg.changeType(colTypeInit -> convert(col.type(), colTypeInit));
           
           if (col.defaultValue() != null)
               colChg.changeDefaultValue(col.defaultValue().toString());
               
           colChg.changeNullable(col.nullable());
   ```

##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnChange;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexChange;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.schemas.table.TablesChange;
+import org.apache.ignite.configuration.tree.NamedListView;
+import org.apache.ignite.internal.schema.ColumnImpl;
+import org.apache.ignite.internal.schema.HashIndexImpl;
+import org.apache.ignite.internal.schema.PartialIndexImpl;
+import org.apache.ignite.internal.schema.PrimaryIndexImpl;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.SortedIndexColumnImpl;
+import org.apache.ignite.internal.schema.SortedIndexImpl;
+import org.apache.ignite.schema.Column;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.IndexColumn;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
+import org.apache.ignite.schema.SortedIndexColumn;
+import org.apache.ignite.schema.TableIndex;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+/**
+ * Configuration to schema and vice versa converter.
+ */
+public class SchemaConfigurationConverter {
+    /** Hash index type. */
+    private static final String HASH_TYPE = "HASH";
+
+    /** Sorted index type. */
+    private static final String SORTED_TYPE = "SORTED";
+
+    /** Partial index type. */
+    private static final String PARTIAL_TYPE = "PARTIAL";
+
+    /** Primary key index type. */
+    private static final String PK_TYPE = "PK";
+
+    /** Types map. */
+    private static Map<String, ColumnType> types = new HashMap<>();
+
+    static {
+        putType(ColumnType.INT8);
+        putType(ColumnType.INT16);
+        putType(ColumnType.INT32);
+        putType(ColumnType.INT64);
+        putType(ColumnType.UINT8);
+        putType(ColumnType.UINT16);
+        putType(ColumnType.UINT32);
+        putType(ColumnType.UINT64);
+        putType(ColumnType.FLOAT);
+        putType(ColumnType.DOUBLE);
+        putType(ColumnType.UUID);
+    }
+
+    private static void putType(ColumnType type) {
+        types.put(type.typeSpec().name(), type);
+    }
+
+    /**
+     * Convert SortedIndexColumn to IndexColumnChange.
+     * @param col IndexColumnChange.
+     * @param colInit IndexColumnChange to fullfill.
+     */
+    public static void convert(SortedIndexColumn col, IndexColumnChange colInit) {
+        colInit.changeName(col.name());
+        colInit.changeAsc(col.asc());
+    }
+
+    /**
+     * Convert IndexColumnView to SortedIndexColumn.
+     *
+     * @param colCfg IndexColumnView.
+     * @return SortedIndexColumn.
+     */
+    public static SortedIndexColumn convert(IndexColumnView colCfg) {
+        return new SortedIndexColumnImpl(colCfg.name(), colCfg.asc());
+    }
+
+    /**
+     * Convert TableIndex to TableIndexChange.
+     *
+     * @param idx TableIndex.
+     * @param idxChg TableIndexChange to fullfill.
+     */
+    public static void convert(TableIndex idx, TableIndexChange idxChg) {
+        idxChg.changeName(idx.name());
+        idxChg.changeType(idx.type());
+        switch (idx.type()) {
+            case HASH_TYPE:
+                HashIndex hashIdx = (HashIndex)idx;
+
+                String[] colNames = hashIdx.columns().stream().map(IndexColumn::name).toArray(String[]::new);
+                idxChg.changeColNames(colNames);
+                break;
+            case PARTIAL_TYPE:
+                PartialIndex partIdx = (PartialIndex)idx;
+
+                idxChg.changeUniq(partIdx.unique());
+                idxChg.changeExpr(partIdx.expr());
+
+                idxChg.changeColumns(colsChg -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : partIdx.columns())
+                        colsChg.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+                break;
+            case SORTED_TYPE:
+                SortedIndex sortIdx = (SortedIndex)idx;
+                idxChg.changeUniq(sortIdx.unique());
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : sortIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+                break;
+            case PK_TYPE:
+                PrimaryIndex primIdx = (PrimaryIndex)idx;
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : primIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+
+                idxChg.changeAffinityColumns(primIdx.affinityColumns().toArray(new String[primIdx.affinityColumns().size()]));
+                break;
+            default:
+                throw new IllegalArgumentException("Unknown index type " + idx.type());
+        }
+    }
+
+    /**
+     * Convert TableIndexView into TableIndex.
+     *
+     * @param idxView TableIndexView.
+     * @return TableIndex.
+     */
+    public static TableIndex convert(TableIndexView idxView) {
+        String name = idxView.name();
+        String type = idxView.type();
+        switch (type) {
+            case "HASH":
+                String[] hashCols = idxView.colNames();
+
+                return new HashIndexImpl(name, hashCols);
+
+            case "SORTED":
+                boolean sortedUniq = idxView.uniq();
+                SortedMap<Integer, SortedIndexColumn> sortedCols = new TreeMap();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    sortedCols.put(Integer.valueOf(key), col);
+                }
+
+                return new SortedIndexImpl(name, new ArrayList<>(sortedCols.values()), sortedUniq);
+
+            case "PARTIAL":
+                boolean partialUniq = idxView.uniq();
+                String expr = idxView.expr();
+                NamedListView<? extends IndexColumnView> colsView = idxView.columns();
+                SortedMap<Integer, SortedIndexColumn> partialCols = new TreeMap<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(colsView.get(key));
+                    partialCols.put(Integer.valueOf(key), col);
+                }
+
+                return new PartialIndexImpl(name, new ArrayList<>(partialCols.values()), partialUniq, expr);
+
+            case "PK":
+                SortedMap<Integer, SortedIndexColumn> cols = new TreeMap<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    cols.put(Integer.valueOf(key), col);
+                }
+                String[] affCols = idxView.affinityColumns();
+
+                return new PrimaryIndexImpl(new ArrayList(cols.values()), List.of(affCols));
+
+            default:
+                throw new IllegalArgumentException("Unknown type " + type);
+        }
+    }
+
+    /**
+     * Convert ColumnType to ColumnTypeChange.
+     *
+     * @param colType ColumnType.
+     * @param colTypeChg ColumnTypeChange to fullfill.
+     */
+    public static void convert(ColumnType colType, ColumnTypeChange colTypeChg) {
+        String typeName = colType.typeSpec().name();
+        if (types.containsKey(typeName))
+            colTypeChg.changeType(typeName);
+        else {
+            colTypeChg.changeType(typeName);
+            switch (typeName) {
+                case "BITMASK":
+                case "BLOB":
+                case "STRING":
+                    ColumnType.VarLenColumnType varLenColType = (ColumnType.VarLenColumnType)colType;
+
+                    colTypeChg.changeLength(varLenColType.length());
+                    break;
+
+                case "DECIMAL":
+                    ColumnType.NumericColumnType numColType = (ColumnType.NumericColumnType)colType;
+
+                    colTypeChg.changePrecision(numColType.precision());
+                    colTypeChg.changeScale(numColType.scale());
+                    break;
+                default:
+                    throw new IllegalArgumentException("Unknown type " + colType.typeSpec().name());
+            }
+        }
+    }
+
+    /**
+     * Convert ColumnTypeView to ColumnType.
+     *
+     * @param colTypeView ColumnTypeView.
+     * @return ColumnType.
+     */
+    public static ColumnType convert(ColumnTypeView colTypeView) {
+        String typeName = colTypeView.type();
+        ColumnType res = types.get(typeName);
+        if (res != null)

Review comment:
       ```suggestion
           ColumnType res = types.get(typeName);
           
           if (res != null)
   ```

##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnChange;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexChange;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.schemas.table.TablesChange;
+import org.apache.ignite.configuration.tree.NamedListView;
+import org.apache.ignite.internal.schema.ColumnImpl;
+import org.apache.ignite.internal.schema.HashIndexImpl;
+import org.apache.ignite.internal.schema.PartialIndexImpl;
+import org.apache.ignite.internal.schema.PrimaryIndexImpl;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.SortedIndexColumnImpl;
+import org.apache.ignite.internal.schema.SortedIndexImpl;
+import org.apache.ignite.schema.Column;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.IndexColumn;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
+import org.apache.ignite.schema.SortedIndexColumn;
+import org.apache.ignite.schema.TableIndex;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+/**
+ * Configuration to schema and vice versa converter.
+ */
+public class SchemaConfigurationConverter {
+    /** Hash index type. */
+    private static final String HASH_TYPE = "HASH";
+
+    /** Sorted index type. */
+    private static final String SORTED_TYPE = "SORTED";
+
+    /** Partial index type. */
+    private static final String PARTIAL_TYPE = "PARTIAL";
+
+    /** Primary key index type. */
+    private static final String PK_TYPE = "PK";
+
+    /** Types map. */
+    private static Map<String, ColumnType> types = new HashMap<>();
+
+    static {
+        putType(ColumnType.INT8);
+        putType(ColumnType.INT16);
+        putType(ColumnType.INT32);
+        putType(ColumnType.INT64);
+        putType(ColumnType.UINT8);
+        putType(ColumnType.UINT16);
+        putType(ColumnType.UINT32);
+        putType(ColumnType.UINT64);
+        putType(ColumnType.FLOAT);
+        putType(ColumnType.DOUBLE);
+        putType(ColumnType.UUID);
+    }
+
+    private static void putType(ColumnType type) {
+        types.put(type.typeSpec().name(), type);
+    }
+
+    /**
+     * Convert SortedIndexColumn to IndexColumnChange.
+     * @param col IndexColumnChange.
+     * @param colInit IndexColumnChange to fullfill.
+     */
+    public static void convert(SortedIndexColumn col, IndexColumnChange colInit) {
+        colInit.changeName(col.name());
+        colInit.changeAsc(col.asc());
+    }
+
+    /**
+     * Convert IndexColumnView to SortedIndexColumn.
+     *
+     * @param colCfg IndexColumnView.
+     * @return SortedIndexColumn.
+     */
+    public static SortedIndexColumn convert(IndexColumnView colCfg) {
+        return new SortedIndexColumnImpl(colCfg.name(), colCfg.asc());
+    }
+
+    /**
+     * Convert TableIndex to TableIndexChange.
+     *
+     * @param idx TableIndex.
+     * @param idxChg TableIndexChange to fullfill.
+     */
+    public static void convert(TableIndex idx, TableIndexChange idxChg) {
+        idxChg.changeName(idx.name());
+        idxChg.changeType(idx.type());
+        switch (idx.type()) {
+            case HASH_TYPE:
+                HashIndex hashIdx = (HashIndex)idx;
+
+                String[] colNames = hashIdx.columns().stream().map(IndexColumn::name).toArray(String[]::new);
+                idxChg.changeColNames(colNames);
+                break;
+            case PARTIAL_TYPE:
+                PartialIndex partIdx = (PartialIndex)idx;
+
+                idxChg.changeUniq(partIdx.unique());
+                idxChg.changeExpr(partIdx.expr());
+
+                idxChg.changeColumns(colsChg -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : partIdx.columns())
+                        colsChg.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+                break;
+            case SORTED_TYPE:
+                SortedIndex sortIdx = (SortedIndex)idx;
+                idxChg.changeUniq(sortIdx.unique());
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : sortIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+                break;
+            case PK_TYPE:
+                PrimaryIndex primIdx = (PrimaryIndex)idx;
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+                    for (SortedIndexColumn col : primIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+
+                idxChg.changeAffinityColumns(primIdx.affinityColumns().toArray(new String[primIdx.affinityColumns().size()]));
+                break;
+            default:
+                throw new IllegalArgumentException("Unknown index type " + idx.type());
+        }
+    }
+
+    /**
+     * Convert TableIndexView into TableIndex.
+     *
+     * @param idxView TableIndexView.
+     * @return TableIndex.
+     */
+    public static TableIndex convert(TableIndexView idxView) {
+        String name = idxView.name();
+        String type = idxView.type();
+        switch (type) {
+            case "HASH":
+                String[] hashCols = idxView.colNames();
+
+                return new HashIndexImpl(name, hashCols);
+
+            case "SORTED":
+                boolean sortedUniq = idxView.uniq();
+                SortedMap<Integer, SortedIndexColumn> sortedCols = new TreeMap();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    sortedCols.put(Integer.valueOf(key), col);
+                }
+
+                return new SortedIndexImpl(name, new ArrayList<>(sortedCols.values()), sortedUniq);
+
+            case "PARTIAL":
+                boolean partialUniq = idxView.uniq();
+                String expr = idxView.expr();
+                NamedListView<? extends IndexColumnView> colsView = idxView.columns();
+                SortedMap<Integer, SortedIndexColumn> partialCols = new TreeMap<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(colsView.get(key));
+                    partialCols.put(Integer.valueOf(key), col);
+                }
+
+                return new PartialIndexImpl(name, new ArrayList<>(partialCols.values()), partialUniq, expr);
+
+            case "PK":
+                SortedMap<Integer, SortedIndexColumn> cols = new TreeMap<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    cols.put(Integer.valueOf(key), col);
+                }
+                String[] affCols = idxView.affinityColumns();
+
+                return new PrimaryIndexImpl(new ArrayList(cols.values()), List.of(affCols));
+
+            default:
+                throw new IllegalArgumentException("Unknown type " + type);
+        }
+    }
+
+    /**
+     * Convert ColumnType to ColumnTypeChange.
+     *
+     * @param colType ColumnType.
+     * @param colTypeChg ColumnTypeChange to fullfill.
+     */
+    public static void convert(ColumnType colType, ColumnTypeChange colTypeChg) {
+        String typeName = colType.typeSpec().name();
+        if (types.containsKey(typeName))
+            colTypeChg.changeType(typeName);
+        else {
+            colTypeChg.changeType(typeName);
+            switch (typeName) {
+                case "BITMASK":
+                case "BLOB":
+                case "STRING":
+                    ColumnType.VarLenColumnType varLenColType = (ColumnType.VarLenColumnType)colType;
+
+                    colTypeChg.changeLength(varLenColType.length());
+                    break;
+
+                case "DECIMAL":
+                    ColumnType.NumericColumnType numColType = (ColumnType.NumericColumnType)colType;
+
+                    colTypeChg.changePrecision(numColType.precision());
+                    colTypeChg.changeScale(numColType.scale());
+                    break;
+                default:
+                    throw new IllegalArgumentException("Unknown type " + colType.typeSpec().name());
+            }
+        }
+    }
+
+    /**
+     * Convert ColumnTypeView to ColumnType.
+     *
+     * @param colTypeView ColumnTypeView.
+     * @return ColumnType.
+     */
+    public static ColumnType convert(ColumnTypeView colTypeView) {
+        String typeName = colTypeView.type();
+        ColumnType res = types.get(typeName);
+        if (res != null)
+            return res;
+        else {
+            switch (typeName) {
+                case "BITMASK":
+                    int bitmaskLen = colTypeView.length();
+
+                    return ColumnType.bitmaskOf(bitmaskLen);
+
+                case "STRING":
+                    int strLen = colTypeView.length();
+
+                    return ColumnType.stringOf(strLen);
+
+                case "BLOB":
+                    int blobLen = colTypeView.length();
+
+                    return ColumnType.blobOf(blobLen);
+
+                case "DECIMAL":
+                    int prec = colTypeView.precision();
+                    int scale = colTypeView.scale();
+
+                    return ColumnType.number(prec, scale);
+
+                default:
+                    throw new IllegalArgumentException("Unknown type " + typeName);
+            }
+        }
+    }
+
+    /**
+     * Convert column to column change.
+     *
+     * @param col Column to convert.
+     * @param colChg Column
+     */
+    public static void convert(Column col, ColumnChange colChg) {
+        colChg.changeName(col.name());
+        colChg.changeType(colTypeInit -> convert(col.type(), colTypeInit));
+        if (col.defaultValue() != null)
+            colChg.changeDefaultValue(col.defaultValue().toString());
+        colChg.changeNullable(col.nullable());
+    }
+
+    /**
+     * Convert column view to Column.
+     *
+     * @param colView Column view.
+     * @return Column.
+     */
+    public static Column convert(ColumnView colView) {
+        String name = colView.name();
+        ColumnType type = convert(colView.type());
+        boolean nullable = colView.nullable();
+        String defValue = colView.defaultValue();
+
+        return new ColumnImpl(name, type, nullable, defValue);
+    }
+
+    /**
+     * Convert schema table to schema table change.
+     *
+     * @param tbl Schema table to convert.
+     * @param tblChg Change to fullfill.
+     */
+    public static void convert(SchemaTable tbl, TableChange tblChg) {
+        tblChg.changeName(tbl.canonicalName());
+        tblChg.changeIndices(idxsChg -> {
+            int idxIdx = 0;
+            for (TableIndex idx : tbl.indices())
+                idxsChg.create(String.valueOf(idxIdx++), idxInit -> convert(idx, idxInit));
+        });
+        tblChg.changeColumns(colsChg -> {
+            int colIdx = 0;
+            for (Column col : tbl.keyColumns())
+                colsChg.create(String.valueOf(colIdx++), colChg -> convert(col, colChg));
+            for (Column col : tbl.valueColumns())
+                colsChg.create(String.valueOf(colIdx++), colChg -> convert(col, colChg));
+        });
+    }
+
+    /**
+     * Convert TableConfiguration to SchemaTable.
+     *
+     * @param tblCfg TableConfiguration to convert.
+     * @return SchemaTable.
+     */
+    public static SchemaTable convert(TableConfiguration tblCfg) {
+        return convert(tblCfg.value());
+    }
+
+    /**
+     * Convert configuration to SchemaTable.
+     *
+     * @param tblView TableView to convert.
+     * @return SchemaTable.
+     */
+    public static SchemaTableImpl convert(TableView tblView) {
+        String canonicalName = tblView.name();
+
+        NamedListView<? extends ColumnView> colsView = tblView.columns();
+        SortedMap<Integer, Column> columns = new TreeMap<>();
+        for (String key : colsView.namedListKeys()) {
+            ColumnView colView = colsView.get(key);
+            Column col = convert(colView);
+            columns.put(Integer.valueOf(key), col);
+        }
+
+        NamedListView<? extends TableIndexView> idxsView = tblView.indices();
+        Map<String, TableIndex> indices = new HashMap<>(idxsView.size());
+        for (String key : idxsView.namedListKeys()) {
+            TableIndexView idxView = idxsView.get(key);
+            TableIndex idx = convert(idxView);
+            indices.put(idx.name(), idx);

Review comment:
       ```suggestion
           NamedListView<? extends TableIndexView> idxsView = tblView.indices();
           
           Map<String, TableIndex> indices = new HashMap<>(idxsView.size());
           
           for (String key : idxsView.namedListKeys()) {
               TableIndex idx = convert(idxsView.get(key));
               
               indices.put(idx.name(), idx);
   ```

##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaDescriptorConverter.java
##########
@@ -0,0 +1,135 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.internal.schema.Column;
+import org.apache.ignite.internal.schema.InvalidTypeException;
+import org.apache.ignite.internal.schema.NativeType;
+import org.apache.ignite.internal.schema.NativeTypes;
+import org.apache.ignite.internal.schema.SchemaDescriptor;
+
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.SchemaTable;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+
+import static org.apache.ignite.internal.schema.NativeTypes.BYTE;
+import static org.apache.ignite.internal.schema.NativeTypes.DOUBLE;
+import static org.apache.ignite.internal.schema.NativeTypes.FLOAT;
+import static org.apache.ignite.internal.schema.NativeTypes.INTEGER;
+import static org.apache.ignite.internal.schema.NativeTypes.LONG;
+import static org.apache.ignite.internal.schema.NativeTypes.SHORT;
+import static org.apache.ignite.internal.schema.NativeTypes.UUID;
+
+/**
+ * Build SchemaDescriptor from SchemaTable internal configuration.
+ */
+public class SchemaDescriptorConverter {
+    /**
+     * Convert ColumnType to NativeType.
+     *
+     * @param colType ColumnType.
+     * @return NativeType.
+     */
+    private static NativeType convert(ColumnType colType) {
+        assert colType != null;
+
+        ColumnType.ColumnTypeSpec type = colType.typeSpec();
+        switch (type) {
+            case INT8:
+                return BYTE;
+
+            case INT16:
+                return SHORT;
+
+            case INT32:
+                return INTEGER;
+
+            case INT64:
+                return LONG;
+
+            case UINT8:
+            case UINT16:
+            case UINT32:
+            case UINT64:
+                throw new UnsupportedOperationException("Unsigned types are not supported yet.");
+
+            case FLOAT:
+                return FLOAT;
+
+            case DOUBLE:
+                return DOUBLE;
+
+            case DECIMAL:
+                ColumnType.NumericColumnType numType = (ColumnType.NumericColumnType)colType;
+
+                return NativeTypes.decimalOf(numType.precision(), numType.scale());
+            case UUID:
+                return UUID;
+
+            case BITMASK:
+                return NativeTypes.bitmaskOf(((ColumnType.VarLenColumnType) colType).length());
+
+            case STRING:
+                return NativeTypes.stringOf(((ColumnType.VarLenColumnType)colType).length());
+
+            case BLOB:
+                return NativeTypes.blobOf(((ColumnType.VarLenColumnType)colType).length());
+
+                default:
+                throw new InvalidTypeException("Unexpected type " + type);
+        }
+    }
+
+    /**
+     * Convert column from public configuration to internal.
+     *
+     * @param colCfg Column to confvert.
+     * @return Internal Column.
+     */
+    private static Column convert(org.apache.ignite.schema.Column colCfg) {
+        return new Column(colCfg.name(), convert(colCfg.type()), colCfg.nullable());
+    }
+
+    /**
+     * Build schema descriptor by SchemaTable.
+     *
+     * @param tblId Table id.
+     * @param schemaVer Schema version.
+     * @param tblCfg SchemaTable.
+     * @return SchemaDescriptor.
+     */
+    public static SchemaDescriptor convert(UUID tblId, int schemaVer, SchemaTable tblCfg) {
+        List<org.apache.ignite.schema.Column> keyColsCfg = new ArrayList<>(tblCfg.keyColumns());
+        Column[] keyCols = new Column[keyColsCfg.size()];
+        for (int i = 0;i < keyCols.length;i++)
+            keyCols[i] = convert(keyColsCfg.get(i));

Review comment:
       ```suggestion
           List<org.apache.ignite.schema.Column> keyColsCfg = new ArrayList<>(tblCfg.keyColumns());
           
           Column[] keyCols = new Column[keyColsCfg.size()];
           
           for (int i = 0;i < keyCols.length;i++)
               keyCols[i] = convert(keyColsCfg.get(i));
   ```

##########
File path: modules/schema/src/test/java/org/apache/ignite/internal/schema/configuration/SchemaDescriptorConverterTest.java
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.internal.schema.Column;
+import org.apache.ignite.internal.schema.Columns;
+import org.apache.ignite.internal.schema.NativeTypeSpec;
+import org.apache.ignite.internal.schema.SchemaDescriptor;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.SchemaBuilders;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.builder.SchemaTableBuilder;
+import org.apache.ignite.schema.builder.TableColumnBuilder;
+import org.junit.jupiter.api.Test;
+
+import java.util.UUID;
+import java.util.function.Function;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Tests for SchemaDescriptorConverter.
+ */
+public class SchemaDescriptorConverterTest {
+    /** Totoal number of columns. */
+    private static final int columns = 14;
+
+    /**
+     * Convert table with complex primary key and check it.
+     */
+    @Test
+    public void testComplexPrimaryIndex() {
+        SchemaTableBuilder bldr = getBuilder(false, false);
+        SchemaTable tblSchm = bldr.withIndex(SchemaBuilders.pkIndex()
+            .addIndexColumn("INT8").done()
+            .addIndexColumn("ID").done()
+            .build()
+        ).build();
+
+        SchemaDescriptor tblDscr = SchemaDescriptorConverter.convert(UUID.randomUUID(), 1, tblSchm);
+
+        assertEquals(2, tblSchm.keyColumns().size());
+        assertEquals(columns - 2, tblSchm.valueColumns().size());
+    }
+
+    /**
+     * Convert table with nullable columns.
+     */
+    @Test
+    public void convertNullable() {
+        testConvert(true);
+    }
+
+    /**
+     * Convert table with non nullable columns.
+     */
+    @Test
+    public void convertTypes() {
+        testConvert(false);
+    }
+
+    /**
+     * Test set of columns.
+     *
+     * @param nullable Nullable flag.
+     */
+    private void testConvert(boolean nullable) {
+        SchemaTable tblSchm = getBuilder(nullable, true).build();
+
+        SchemaDescriptor tblDscr = SchemaDescriptorConverter.convert(UUID.randomUUID(), 1, tblSchm);
+
+        assertEquals(1, tblDscr.keyColumns().length());
+        testCol(tblDscr.keyColumns(), "ID", NativeTypeSpec.UUID, nullable);
+
+        assertEquals(columns - 1, tblDscr.valueColumns().length());
+        testCol(tblDscr.valueColumns(), "INT8", NativeTypeSpec.BYTE, nullable);
+        testCol(tblDscr.valueColumns(), "INT16", NativeTypeSpec.SHORT, nullable);
+        testCol(tblDscr.valueColumns(), "INT32", NativeTypeSpec.INTEGER, nullable);
+        testCol(tblDscr.valueColumns(), "INT64", NativeTypeSpec.LONG, nullable);
+        testCol(tblDscr.valueColumns(), "FLOAT", NativeTypeSpec.FLOAT, nullable);
+        testCol(tblDscr.valueColumns(), "DOUBLE", NativeTypeSpec.DOUBLE, nullable);
+        testCol(tblDscr.valueColumns(), "UUID", NativeTypeSpec.UUID, nullable);
+        testCol(tblDscr.valueColumns(), "STRING", NativeTypeSpec.STRING, nullable);
+        testCol(tblDscr.valueColumns(), "STRING_FS10", NativeTypeSpec.STRING, nullable);
+        testCol(tblDscr.valueColumns(), "BLOB", NativeTypeSpec.BYTES, nullable);
+        testCol(tblDscr.valueColumns(), "DECIMAL", NativeTypeSpec.DECIMAL, nullable);
+        testCol(tblDscr.valueColumns(), "BITMASK_FS10", NativeTypeSpec.BITMASK, nullable);
+    }
+
+    /**
+     * Get SchemaTableBuilder with default table.
+     *
+     * @param nullable If all columns should be nullable.
+     * @param withPk If builder should contains primary key index.
+     * @return SchemaTableBuilder.
+     */
+    private SchemaTableBuilder getBuilder(boolean nullable, boolean withPk) {
+        Function<TableColumnBuilder, org.apache.ignite.schema.Column> postProcess = builder -> {
+            if (nullable)
+                builder.asNullable();
+            else
+                builder.asNonNull();
+            return builder.build();
+        };
+
+        SchemaTableBuilder res = SchemaBuilders.tableBuilder("SCHEMA", "TABLE")
+            .columns(
+                postProcess.apply(SchemaBuilders.column("ID", ColumnType.UUID)),
+                postProcess.apply(SchemaBuilders.column("INT8", ColumnType.INT8)),
+                postProcess.apply(SchemaBuilders.column("INT16", ColumnType.INT16)),
+                postProcess.apply(SchemaBuilders.column("INT32", ColumnType.INT32)),
+                postProcess.apply(SchemaBuilders.column("INT64", ColumnType.INT64)),
+                postProcess.apply(SchemaBuilders.column("FLOAT", ColumnType.FLOAT)),
+                postProcess.apply(SchemaBuilders.column("DOUBLE", ColumnType.DOUBLE)),
+                postProcess.apply(SchemaBuilders.column("UUID", ColumnType.UUID)),
+                postProcess.apply(SchemaBuilders.column("STRING", ColumnType.string())),
+                postProcess.apply(SchemaBuilders.column("STRING_FS10", ColumnType.stringOf(10))),
+                postProcess.apply(SchemaBuilders.column("BLOB", ColumnType.blobOf())),
+                postProcess.apply(SchemaBuilders.column("BLOB_FS10", ColumnType.blobOf(10))),
+                postProcess.apply(SchemaBuilders.column("DECIMAL", ColumnType.number(1,1))),
+                postProcess.apply(SchemaBuilders.column("BITMASK_FS10", ColumnType.bitmaskOf(10)))
+                // TODO: uncomment after unsigned types available
+                // postProcess.apply(SchemaBuilders.column("UINT8", ColumnType.UINT8)),
+                // postProcess.apply(SchemaBuilders.column("UINT16", ColumnType.UINT16)),
+                // postProcess.apply(SchemaBuilders.column("UINT32", ColumnType.UINT32)),
+                // postProcess.apply(SchemaBuilders.column("UINT64", ColumnType.UINT64)),
+            );
+        if (withPk)
+            res.withPrimaryKey("ID");
+
+        return res;
+    }
+
+    /**
+     * Check specified column to match other parameters.
+     *
+     * @param cols Columns to test.
+     * @param name Expected column name.
+     * @param type Expected column type.
+     * @param nullable Expected column nullable flag.
+     */
+    private static void testCol(Columns cols, String name, NativeTypeSpec type, boolean nullable) {
+        int idx = cols.columnIndex(name);
+        Column col = cols.column(idx);
+
+        assertEquals(name, col.name());
+        assertEquals(type.name(), col.type().spec().name());
+        assertEquals(nullable, col.nullable());
+        if (col.type().spec().fixedLength())
+            assertTrue(col.type().sizeInBytes() >= 0);

Review comment:
       ```suggestion
   
           if (col.type().spec().fixedLength())
               assertTrue(col.type().sizeInBytes() >= 0);
   ```




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

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



[GitHub] [ignite-3] AMashenkov merged pull request #132: IGNITE-14290: Schema configuration API implementation.

Posted by GitBox <gi...@apache.org>.
AMashenkov merged pull request #132:
URL: https://github.com/apache/ignite-3/pull/132


   


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

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



[GitHub] [ignite-3] AMashenkov commented on a change in pull request #132: GG-14290: SchemaDescriptor and ConfigurationSchema converters.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #132:
URL: https://github.com/apache/ignite-3/pull/132#discussion_r635025006



##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaDescriptorConverter.java
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.internal.schema.Bitmask;
+import org.apache.ignite.internal.schema.Column;
+import org.apache.ignite.internal.schema.InvalidTypeException;
+import org.apache.ignite.internal.schema.NativeType;
+import org.apache.ignite.internal.schema.NativeTypeSpec;
+import org.apache.ignite.internal.schema.NumericNativeType;
+import org.apache.ignite.internal.schema.SchemaDescriptor;
+
+import org.apache.ignite.internal.schema.VarlenNativeType;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.SchemaTable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import static org.apache.ignite.internal.schema.NativeType.BYTE;
+import static org.apache.ignite.internal.schema.NativeType.DOUBLE;
+import static org.apache.ignite.internal.schema.NativeType.FLOAT;
+import static org.apache.ignite.internal.schema.NativeType.INTEGER;
+import static org.apache.ignite.internal.schema.NativeType.LONG;
+import static org.apache.ignite.internal.schema.NativeType.SHORT;
+import static org.apache.ignite.internal.schema.NativeType.UUID;
+
+/** Build SchemaDescriptor from SchemaTable internal configuration. */
+public class SchemaDescriptorConverter {
+    /**
+     * Convert ColumnType to NativeType.
+     *
+     * @param colType ColumnType.
+     * @return NativeType.
+     */
+    private static NativeType convert(ColumnType colType) {
+        assert colType != null;
+
+        ColumnType.ColumnTypeSpec type = colType.typeSpec();

Review comment:
       ```suggestion
           ColumnType.ColumnTypeSpec type = colType.typeSpec();
           
   ```




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

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



[GitHub] [ignite-3] AMashenkov commented on a change in pull request #132: GG-14290: SchemaDescriptor and ConfigurationSchema converters.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #132:
URL: https://github.com/apache/ignite-3/pull/132#discussion_r635044650



##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/builder/SchemaTableBuilderImpl.java
##########
@@ -116,18 +121,20 @@ else if (PRIMARY_KEY_INDEX_NAME.equals(index.name()))
     /**
      * Validate indices.
      */
-    private void validateIndices() {
-        assert indices.values().stream()
+    public static void validateIndices(Collection<TableIndex> indices, Collection<Column> columns) {
+        Set<String> colNames = columns.stream().map(Column::name).collect(Collectors.toSet());
+        assert indices.stream()

Review comment:
       ```suggestion
           Set<String> colNames = columns.stream().map(Column::name).collect(Collectors.toSet());
           
           assert indices.stream()
   ```

##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/builder/SchemaTableBuilderImpl.java
##########
@@ -116,18 +121,20 @@ else if (PRIMARY_KEY_INDEX_NAME.equals(index.name()))
     /**
      * Validate indices.
      */
-    private void validateIndices() {
-        assert indices.values().stream()
+    public static void validateIndices(Collection<TableIndex> indices, Collection<Column> columns) {
+        Set<String> colNames = columns.stream().map(Column::name).collect(Collectors.toSet());
+        assert indices.stream()
             .filter(ColumnarIndex.class::isInstance)
             .map(ColumnarIndex.class::cast)
             .flatMap(idx -> idx.columns().stream())
             .map(IndexColumn::name)
-            .allMatch(columns::containsKey) : "Index column doesn't exists in schema.";
+            .allMatch(colNames::contains) : "Index column doesn't exists in schema.";
 
-        assert indices.containsKey(PRIMARY_KEY_INDEX_NAME) : "Primary key index is not configured.";
-        assert !((PrimaryIndex)indices.get(PRIMARY_KEY_INDEX_NAME)).affinityColumns().isEmpty() : "Primary key must have one affinity column at least.";
+        TableIndex pkIdx = indices.stream().filter(idx -> PRIMARY_KEY_INDEX_NAME.equals(idx.name())).findAny().orElse(null);
+        assert pkIdx != null : "Primary key index is not configured.";

Review comment:
       ```suggestion
           TableIndex pkIdx = indices.stream().filter(idx -> PRIMARY_KEY_INDEX_NAME.equals(idx.name())).findAny().orElse(null);
           
           assert pkIdx != null : "Primary key index is not configured.";
   ```




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

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



[GitHub] [ignite-3] AMashenkov commented on a change in pull request #132: GG-14290: SchemaDescriptor and ConfigurationSchema converters.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #132:
URL: https://github.com/apache/ignite-3/pull/132#discussion_r635034173



##########
File path: modules/schema/src/test/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverterTest.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.ConfigurationRegistry;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableValidator;
+import org.apache.ignite.configuration.schemas.table.TablesConfiguration;
+import org.apache.ignite.internal.schema.builder.HashIndexBuilderTest;
+import org.apache.ignite.internal.schema.builder.PartialIndexBuilderTest;
+import org.apache.ignite.internal.schema.builder.PrimaryKeyBuilderTest;
+import org.apache.ignite.internal.schema.builder.SchemaTableBuilderTest;
+import org.apache.ignite.internal.schema.builder.SortedIndexBuilderTest;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Collections;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.ignite.internal.schema.builder.SchemaTableBuilderTest.TBL;
+
+/** SchmConfigurationConverter tests. */

Review comment:
       SchemaConfigurationConverter ?




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

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



[GitHub] [ignite-3] AMashenkov commented on a change in pull request #132: GG-14290: SchemaDescriptor and ConfigurationSchema converters.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #132:
URL: https://github.com/apache/ignite-3/pull/132#discussion_r635019424



##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnChange;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexChange;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.schemas.table.TablesChange;
+import org.apache.ignite.configuration.tree.NamedListView;
+import org.apache.ignite.internal.schema.ColumnImpl;
+import org.apache.ignite.internal.schema.HashIndexImpl;
+import org.apache.ignite.internal.schema.PartialIndexImpl;
+import org.apache.ignite.internal.schema.PrimaryIndexImpl;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.SortedIndexColumnImpl;
+import org.apache.ignite.internal.schema.SortedIndexImpl;
+import org.apache.ignite.schema.Column;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.IndexColumn;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
+import org.apache.ignite.schema.SortedIndexColumn;
+import org.apache.ignite.schema.TableIndex;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/** Configuration to schema and vice versa converter. */
+public class SchemaConfigurationConverter {
+    /** Hash index type. */
+    private static final String HASH_TYPE = "HASH";
+
+    /** Sorted index type. */
+    private static final String SORTED_TYPE = "SORTED";
+
+    /** Partial index type. */
+    private static final String PARTIAL_TYPE = "PARTIAL";
+
+    /** Primary key index type. */
+    private static final String PK_TYPE = "PK";
+
+    /** Types map. */
+    private static Map<String, ColumnType> types = new HashMap<>();
+
+    static {
+        putType(ColumnType.INT8);
+        putType(ColumnType.INT16);
+        putType(ColumnType.INT32);
+        putType(ColumnType.INT64);
+        putType(ColumnType.UINT8);
+        putType(ColumnType.UINT16);
+        putType(ColumnType.UINT32);
+        putType(ColumnType.UINT64);
+        putType(ColumnType.FLOAT);
+        putType(ColumnType.DOUBLE);
+        putType(ColumnType.UUID);
+        putType(ColumnType.string());
+        putType(ColumnType.blobOf());
+        // TODO; handle length for some types
+        //putType(ColumnType.bitmaskOf());
+    }
+
+    private static void putType(ColumnType type) {
+        types.put(type.typeSpec().name(), type);
+    }
+
+    /**
+     * Convert SortedIndexColumn to IndexColumnChange.
+     * @param col IndexColumnChange.
+     * @param colInit IndexColumnChange to fullfill.
+     */
+    public static void convert(SortedIndexColumn col, IndexColumnChange colInit) {
+        colInit.changeName(col.name());
+        colInit.changeAsc(col.asc());
+    }
+
+    /**
+     * Convert IndexColumnView to SortedIndexColumn.
+     *
+     * @param colCfg IndexColumnView.
+     * @return SortedIndexColumn.
+     */
+    public static SortedIndexColumn convert(IndexColumnView colCfg) {
+        return new SortedIndexColumnImpl(colCfg.name(), colCfg.asc());
+    }
+
+    /**
+     * Convert TableIndex to TableIndexChange.
+     *
+     * @param idx TableIndex.
+     * @param idxChg TableIndexChange to fullfill.
+     */
+    public static void convert(TableIndex idx, TableIndexChange idxChg) {
+        idxChg.changeName(idx.name());
+        idxChg.changeType(idx.type());
+
+        if (HASH_TYPE.equals(idx.type())) {
+            HashIndex hashIdx = (HashIndex)idx;
+
+            String[] colNames = hashIdx.columns().stream().map(IndexColumn::name).toArray(String[]::new);
+            idxChg.changeColNames(colNames);
+        }
+        else if (PARTIAL_TYPE.equals(idx.type())) {
+            PartialIndex partIdx = (PartialIndex)idx;
+
+            idxChg.changeUniq(partIdx.unique());
+            idxChg.changeExpr(partIdx.expr());
+
+            idxChg.changeColumns(colsChg -> {
+                for (SortedIndexColumn col : partIdx.columns())
+                    colsChg.create(col.name(), colInit -> convert(col, colInit));
+            });
+
+        }
+        else if (SORTED_TYPE.equals(idx.type())) {
+            SortedIndex sortIdx = (SortedIndex)idx;
+            idxChg.changeUniq(sortIdx.unique());
+
+            idxChg.changeColumns(colsInit -> {
+                for (SortedIndexColumn col : sortIdx.columns())
+                    colsInit.create(col.name(), colInit -> convert(col, colInit));
+            });
+        }
+        else if (PK_TYPE.equals(idx.type())) {
+            PrimaryIndex primIdx = (PrimaryIndex)idx;
+
+            idxChg.changeColumns(colsInit -> {
+                for (SortedIndexColumn col : primIdx.columns())
+                    colsInit.create(col.name(), colInit -> convert(col, colInit));
+            });
+
+            idxChg.changeAffinityColumns(primIdx.affinityColumns().toArray(new String[primIdx.affinityColumns().size()]));
+        }
+        else throw new IllegalArgumentException("Unknown index type " + idx.type());
+    }
+
+    /**
+     * Convert TableIndexView into TableIndex.
+     *
+     * @param idxView TableIndexView.
+     * @return TableIndex.
+     */
+    public static TableIndex convert(TableIndexView idxView) {
+        String name = idxView.name();
+        String type = idxView.type();
+
+        if (type.equals("HASH")) {
+            String[] cols = idxView.colNames();
+
+            return new HashIndexImpl(name, cols);
+        }
+        else if (type.equals("SORTED") || type.equals("PARTIAL") || type.equals("PK")) {
+            if (type.equals("PARTIAL")) {
+                boolean uniq = idxView.uniq();
+                String expr = idxView.expr();
+                NamedListView<? extends IndexColumnView> colsView = idxView.columns();
+                List<SortedIndexColumn> cols = new ArrayList<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(colsView.get(key));
+                    cols.add(col);
+                }
+
+                return new PartialIndexImpl(name, cols, uniq, expr);
+            }
+            if (type.equals("SORTED")) {
+                boolean uniq = idxView.uniq();
+                List<SortedIndexColumn> cols = new ArrayList<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    cols.add(col);
+                }
+
+                return new SortedIndexImpl(name, cols, uniq);
+            }
+            if (type.equals("PK")) {
+                List<SortedIndexColumn> cols = new ArrayList<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    cols.add(col);
+                }
+                String[] affCols = idxView.colNames();
+
+                return new PrimaryIndexImpl(cols, Arrays.stream(affCols).collect(Collectors.toList()));
+            }
+
+            return null;
+        }
+        else
+            throw new IllegalArgumentException("Unknown type " + type);
+    }
+
+    /**
+     * Convert ColumnType to ColumnTypeChange.
+     *
+     * @param colType ColumnType.
+     * @param colTypeChg ColumnTypeChange to fullfill.
+     */
+    public static void convert(ColumnType colType, ColumnTypeChange colTypeChg) {
+        colTypeChg.changeType(colType.typeSpec().name());
+        // TODO varlen types
+    }
+
+    /**
+     * Convert ColumnTypeView to ColumnType.
+     *
+     * @param colTypeView ColumnTypeView.
+     * @return ColumnType.
+     */
+    public static ColumnType convert(ColumnTypeView colTypeView) {
+        String typeName = colTypeView.type();
+        ColumnType res = types.get(typeName);
+        if (res != null)
+            return res;
+        else
+            // TODO varlen types
+            throw new IllegalArgumentException();
+    }
+
+    /**
+     * Convert column to column change.
+     *
+     * @param col Column to convert.
+     * @param colChg Column
+     */
+    public static void convert(Column col, ColumnChange colChg) {
+        colChg.changeName(col.name());
+        colChg.changeType(colTypeInit -> convert(col.type(), colTypeInit));
+        if (col.defaultValue() != null)
+            colChg.changeDefaultValue(col.defaultValue().toString()); // TODO: specify "default type" type
+        colChg.changeNullable(col.nullable());
+    }
+
+    /**
+     * Convert column view to Column.
+     *
+     * @param colView Column view.
+     * @return Column.
+     */
+    public static Column convert(ColumnView colView) {
+        String name = colView.name();
+        ColumnType type = convert(colView.type());
+        boolean nullable = colView.nullable();
+        String defValue = colView.defaultValue();
+
+        return new ColumnImpl(name, type, nullable, defValue);
+    }
+
+    /**
+     * Convert schema table to schema table change.
+     *
+     * @param tbl Schema table to convert.
+     * @param tblChg Change to fullfill.
+     */
+    public static void convert(SchemaTable tbl, TableChange tblChg) {
+        tblChg.changeName(tbl.canonicalName());
+        tblChg.changeIndices(idxsChg -> {
+            for (TableIndex idx : tbl.indices())
+                idxsChg.create(idx.name(), idxInit -> convert(idx, idxInit));
+        });
+        tblChg.changeColumns(colsChg -> {
+            for (Column col : tbl.keyColumns())
+                colsChg.create(col.name(), colChg -> convert(col, colChg));
+            for (Column col : tbl.valueColumns())

Review comment:
       ```suggestion
                   colsChg.create(col.name(), colChg -> convert(col, colChg));
                   
               for (Column col : tbl.valueColumns())
   ```




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

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



[GitHub] [ignite-3] AMashenkov commented on a change in pull request #132: GG-14290: SchemaDescriptor and ConfigurationSchema converters.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #132:
URL: https://github.com/apache/ignite-3/pull/132#discussion_r635022730



##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnChange;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexChange;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.schemas.table.TablesChange;
+import org.apache.ignite.configuration.tree.NamedListView;
+import org.apache.ignite.internal.schema.ColumnImpl;
+import org.apache.ignite.internal.schema.HashIndexImpl;
+import org.apache.ignite.internal.schema.PartialIndexImpl;
+import org.apache.ignite.internal.schema.PrimaryIndexImpl;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.SortedIndexColumnImpl;
+import org.apache.ignite.internal.schema.SortedIndexImpl;
+import org.apache.ignite.schema.Column;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.IndexColumn;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
+import org.apache.ignite.schema.SortedIndexColumn;
+import org.apache.ignite.schema.TableIndex;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/** Configuration to schema and vice versa converter. */
+public class SchemaConfigurationConverter {
+    /** Hash index type. */
+    private static final String HASH_TYPE = "HASH";
+
+    /** Sorted index type. */
+    private static final String SORTED_TYPE = "SORTED";
+
+    /** Partial index type. */
+    private static final String PARTIAL_TYPE = "PARTIAL";
+
+    /** Primary key index type. */
+    private static final String PK_TYPE = "PK";
+
+    /** Types map. */
+    private static Map<String, ColumnType> types = new HashMap<>();
+
+    static {
+        putType(ColumnType.INT8);
+        putType(ColumnType.INT16);
+        putType(ColumnType.INT32);
+        putType(ColumnType.INT64);
+        putType(ColumnType.UINT8);
+        putType(ColumnType.UINT16);
+        putType(ColumnType.UINT32);
+        putType(ColumnType.UINT64);
+        putType(ColumnType.FLOAT);
+        putType(ColumnType.DOUBLE);
+        putType(ColumnType.UUID);
+        putType(ColumnType.string());
+        putType(ColumnType.blobOf());
+        // TODO; handle length for some types
+        //putType(ColumnType.bitmaskOf());
+    }
+
+    private static void putType(ColumnType type) {
+        types.put(type.typeSpec().name(), type);
+    }
+
+    /**
+     * Convert SortedIndexColumn to IndexColumnChange.
+     * @param col IndexColumnChange.
+     * @param colInit IndexColumnChange to fullfill.
+     */
+    public static void convert(SortedIndexColumn col, IndexColumnChange colInit) {
+        colInit.changeName(col.name());
+        colInit.changeAsc(col.asc());
+    }
+
+    /**
+     * Convert IndexColumnView to SortedIndexColumn.
+     *
+     * @param colCfg IndexColumnView.
+     * @return SortedIndexColumn.
+     */
+    public static SortedIndexColumn convert(IndexColumnView colCfg) {
+        return new SortedIndexColumnImpl(colCfg.name(), colCfg.asc());
+    }
+
+    /**
+     * Convert TableIndex to TableIndexChange.
+     *
+     * @param idx TableIndex.
+     * @param idxChg TableIndexChange to fullfill.
+     */
+    public static void convert(TableIndex idx, TableIndexChange idxChg) {
+        idxChg.changeName(idx.name());
+        idxChg.changeType(idx.type());
+
+        if (HASH_TYPE.equals(idx.type())) {
+            HashIndex hashIdx = (HashIndex)idx;
+
+            String[] colNames = hashIdx.columns().stream().map(IndexColumn::name).toArray(String[]::new);
+            idxChg.changeColNames(colNames);
+        }
+        else if (PARTIAL_TYPE.equals(idx.type())) {
+            PartialIndex partIdx = (PartialIndex)idx;
+
+            idxChg.changeUniq(partIdx.unique());
+            idxChg.changeExpr(partIdx.expr());
+
+            idxChg.changeColumns(colsChg -> {
+                for (SortedIndexColumn col : partIdx.columns())
+                    colsChg.create(col.name(), colInit -> convert(col, colInit));
+            });
+
+        }
+        else if (SORTED_TYPE.equals(idx.type())) {
+            SortedIndex sortIdx = (SortedIndex)idx;
+            idxChg.changeUniq(sortIdx.unique());
+
+            idxChg.changeColumns(colsInit -> {
+                for (SortedIndexColumn col : sortIdx.columns())
+                    colsInit.create(col.name(), colInit -> convert(col, colInit));
+            });
+        }
+        else if (PK_TYPE.equals(idx.type())) {
+            PrimaryIndex primIdx = (PrimaryIndex)idx;
+
+            idxChg.changeColumns(colsInit -> {
+                for (SortedIndexColumn col : primIdx.columns())
+                    colsInit.create(col.name(), colInit -> convert(col, colInit));
+            });
+
+            idxChg.changeAffinityColumns(primIdx.affinityColumns().toArray(new String[primIdx.affinityColumns().size()]));
+        }
+        else throw new IllegalArgumentException("Unknown index type " + idx.type());
+    }
+
+    /**
+     * Convert TableIndexView into TableIndex.
+     *
+     * @param idxView TableIndexView.
+     * @return TableIndex.
+     */
+    public static TableIndex convert(TableIndexView idxView) {
+        String name = idxView.name();
+        String type = idxView.type();
+
+        if (type.equals("HASH")) {
+            String[] cols = idxView.colNames();
+
+            return new HashIndexImpl(name, cols);
+        }
+        else if (type.equals("SORTED") || type.equals("PARTIAL") || type.equals("PK")) {
+            if (type.equals("PARTIAL")) {
+                boolean uniq = idxView.uniq();
+                String expr = idxView.expr();
+                NamedListView<? extends IndexColumnView> colsView = idxView.columns();
+                List<SortedIndexColumn> cols = new ArrayList<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(colsView.get(key));
+                    cols.add(col);
+                }
+
+                return new PartialIndexImpl(name, cols, uniq, expr);
+            }
+            if (type.equals("SORTED")) {
+                boolean uniq = idxView.uniq();
+                List<SortedIndexColumn> cols = new ArrayList<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    cols.add(col);
+                }
+
+                return new SortedIndexImpl(name, cols, uniq);
+            }
+            if (type.equals("PK")) {
+                List<SortedIndexColumn> cols = new ArrayList<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    cols.add(col);
+                }
+                String[] affCols = idxView.colNames();
+
+                return new PrimaryIndexImpl(cols, Arrays.stream(affCols).collect(Collectors.toList()));
+            }
+
+            return null;
+        }
+        else
+            throw new IllegalArgumentException("Unknown type " + type);
+    }
+
+    /**
+     * Convert ColumnType to ColumnTypeChange.
+     *
+     * @param colType ColumnType.
+     * @param colTypeChg ColumnTypeChange to fullfill.
+     */
+    public static void convert(ColumnType colType, ColumnTypeChange colTypeChg) {
+        colTypeChg.changeType(colType.typeSpec().name());
+        // TODO varlen types
+    }
+
+    /**
+     * Convert ColumnTypeView to ColumnType.
+     *
+     * @param colTypeView ColumnTypeView.
+     * @return ColumnType.
+     */
+    public static ColumnType convert(ColumnTypeView colTypeView) {
+        String typeName = colTypeView.type();
+        ColumnType res = types.get(typeName);
+        if (res != null)
+            return res;
+        else
+            // TODO varlen types
+            throw new IllegalArgumentException();
+    }
+
+    /**
+     * Convert column to column change.
+     *
+     * @param col Column to convert.
+     * @param colChg Column
+     */
+    public static void convert(Column col, ColumnChange colChg) {
+        colChg.changeName(col.name());
+        colChg.changeType(colTypeInit -> convert(col.type(), colTypeInit));
+        if (col.defaultValue() != null)
+            colChg.changeDefaultValue(col.defaultValue().toString()); // TODO: specify "default type" type
+        colChg.changeNullable(col.nullable());
+    }
+
+    /**
+     * Convert column view to Column.
+     *
+     * @param colView Column view.
+     * @return Column.
+     */
+    public static Column convert(ColumnView colView) {
+        String name = colView.name();
+        ColumnType type = convert(colView.type());
+        boolean nullable = colView.nullable();
+        String defValue = colView.defaultValue();
+
+        return new ColumnImpl(name, type, nullable, defValue);
+    }
+
+    /**
+     * Convert schema table to schema table change.
+     *
+     * @param tbl Schema table to convert.
+     * @param tblChg Change to fullfill.
+     */
+    public static void convert(SchemaTable tbl, TableChange tblChg) {
+        tblChg.changeName(tbl.canonicalName());
+        tblChg.changeIndices(idxsChg -> {
+            for (TableIndex idx : tbl.indices())
+                idxsChg.create(idx.name(), idxInit -> convert(idx, idxInit));
+        });
+        tblChg.changeColumns(colsChg -> {
+            for (Column col : tbl.keyColumns())
+                colsChg.create(col.name(), colChg -> convert(col, colChg));
+            for (Column col : tbl.valueColumns())
+                colsChg.create(col.name(), colChg -> convert(col, colChg));
+        });
+    }
+
+    public static SchemaTable convert(TableConfiguration tblCfg) {
+        return convert(tblCfg.value());
+    }
+
+    /**
+     * Convert configuration to SchemaTable.
+     *
+     * @param tblView SchmTableView to convert.
+     * @return SchemaTable.
+     */
+    public static SchemaTableImpl convert(TableView tblView) {
+        String canonicalName = tblView.name();
+
+        NamedListView<? extends ColumnView> colsView = tblView.columns();
+        LinkedHashMap<String, Column> columns = new LinkedHashMap<>(colsView.size());
+        for (String key : colsView.namedListKeys()) {
+            ColumnView colView = colsView.get(key);
+            Column col = convert(colView);
+            columns.put(col.name(), col);

Review comment:
       Empty line.

##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnChange;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexChange;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.schemas.table.TablesChange;
+import org.apache.ignite.configuration.tree.NamedListView;
+import org.apache.ignite.internal.schema.ColumnImpl;
+import org.apache.ignite.internal.schema.HashIndexImpl;
+import org.apache.ignite.internal.schema.PartialIndexImpl;
+import org.apache.ignite.internal.schema.PrimaryIndexImpl;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.SortedIndexColumnImpl;
+import org.apache.ignite.internal.schema.SortedIndexImpl;
+import org.apache.ignite.schema.Column;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.IndexColumn;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
+import org.apache.ignite.schema.SortedIndexColumn;
+import org.apache.ignite.schema.TableIndex;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/** Configuration to schema and vice versa converter. */
+public class SchemaConfigurationConverter {
+    /** Hash index type. */
+    private static final String HASH_TYPE = "HASH";
+
+    /** Sorted index type. */
+    private static final String SORTED_TYPE = "SORTED";
+
+    /** Partial index type. */
+    private static final String PARTIAL_TYPE = "PARTIAL";
+
+    /** Primary key index type. */
+    private static final String PK_TYPE = "PK";
+
+    /** Types map. */
+    private static Map<String, ColumnType> types = new HashMap<>();
+
+    static {
+        putType(ColumnType.INT8);
+        putType(ColumnType.INT16);
+        putType(ColumnType.INT32);
+        putType(ColumnType.INT64);
+        putType(ColumnType.UINT8);
+        putType(ColumnType.UINT16);
+        putType(ColumnType.UINT32);
+        putType(ColumnType.UINT64);
+        putType(ColumnType.FLOAT);
+        putType(ColumnType.DOUBLE);
+        putType(ColumnType.UUID);
+        putType(ColumnType.string());
+        putType(ColumnType.blobOf());
+        // TODO; handle length for some types
+        //putType(ColumnType.bitmaskOf());
+    }
+
+    private static void putType(ColumnType type) {
+        types.put(type.typeSpec().name(), type);
+    }
+
+    /**
+     * Convert SortedIndexColumn to IndexColumnChange.
+     * @param col IndexColumnChange.
+     * @param colInit IndexColumnChange to fullfill.
+     */
+    public static void convert(SortedIndexColumn col, IndexColumnChange colInit) {
+        colInit.changeName(col.name());
+        colInit.changeAsc(col.asc());
+    }
+
+    /**
+     * Convert IndexColumnView to SortedIndexColumn.
+     *
+     * @param colCfg IndexColumnView.
+     * @return SortedIndexColumn.
+     */
+    public static SortedIndexColumn convert(IndexColumnView colCfg) {
+        return new SortedIndexColumnImpl(colCfg.name(), colCfg.asc());
+    }
+
+    /**
+     * Convert TableIndex to TableIndexChange.
+     *
+     * @param idx TableIndex.
+     * @param idxChg TableIndexChange to fullfill.
+     */
+    public static void convert(TableIndex idx, TableIndexChange idxChg) {
+        idxChg.changeName(idx.name());
+        idxChg.changeType(idx.type());
+
+        if (HASH_TYPE.equals(idx.type())) {
+            HashIndex hashIdx = (HashIndex)idx;
+
+            String[] colNames = hashIdx.columns().stream().map(IndexColumn::name).toArray(String[]::new);
+            idxChg.changeColNames(colNames);
+        }
+        else if (PARTIAL_TYPE.equals(idx.type())) {
+            PartialIndex partIdx = (PartialIndex)idx;
+
+            idxChg.changeUniq(partIdx.unique());
+            idxChg.changeExpr(partIdx.expr());
+
+            idxChg.changeColumns(colsChg -> {
+                for (SortedIndexColumn col : partIdx.columns())
+                    colsChg.create(col.name(), colInit -> convert(col, colInit));
+            });
+
+        }
+        else if (SORTED_TYPE.equals(idx.type())) {
+            SortedIndex sortIdx = (SortedIndex)idx;
+            idxChg.changeUniq(sortIdx.unique());
+
+            idxChg.changeColumns(colsInit -> {
+                for (SortedIndexColumn col : sortIdx.columns())
+                    colsInit.create(col.name(), colInit -> convert(col, colInit));
+            });
+        }
+        else if (PK_TYPE.equals(idx.type())) {
+            PrimaryIndex primIdx = (PrimaryIndex)idx;
+
+            idxChg.changeColumns(colsInit -> {
+                for (SortedIndexColumn col : primIdx.columns())
+                    colsInit.create(col.name(), colInit -> convert(col, colInit));
+            });
+
+            idxChg.changeAffinityColumns(primIdx.affinityColumns().toArray(new String[primIdx.affinityColumns().size()]));
+        }
+        else throw new IllegalArgumentException("Unknown index type " + idx.type());
+    }
+
+    /**
+     * Convert TableIndexView into TableIndex.
+     *
+     * @param idxView TableIndexView.
+     * @return TableIndex.
+     */
+    public static TableIndex convert(TableIndexView idxView) {
+        String name = idxView.name();
+        String type = idxView.type();
+
+        if (type.equals("HASH")) {
+            String[] cols = idxView.colNames();
+
+            return new HashIndexImpl(name, cols);
+        }
+        else if (type.equals("SORTED") || type.equals("PARTIAL") || type.equals("PK")) {
+            if (type.equals("PARTIAL")) {
+                boolean uniq = idxView.uniq();
+                String expr = idxView.expr();
+                NamedListView<? extends IndexColumnView> colsView = idxView.columns();
+                List<SortedIndexColumn> cols = new ArrayList<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(colsView.get(key));
+                    cols.add(col);
+                }
+
+                return new PartialIndexImpl(name, cols, uniq, expr);
+            }
+            if (type.equals("SORTED")) {
+                boolean uniq = idxView.uniq();
+                List<SortedIndexColumn> cols = new ArrayList<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    cols.add(col);
+                }
+
+                return new SortedIndexImpl(name, cols, uniq);
+            }
+            if (type.equals("PK")) {
+                List<SortedIndexColumn> cols = new ArrayList<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    cols.add(col);
+                }
+                String[] affCols = idxView.colNames();
+
+                return new PrimaryIndexImpl(cols, Arrays.stream(affCols).collect(Collectors.toList()));
+            }
+
+            return null;
+        }
+        else
+            throw new IllegalArgumentException("Unknown type " + type);
+    }
+
+    /**
+     * Convert ColumnType to ColumnTypeChange.
+     *
+     * @param colType ColumnType.
+     * @param colTypeChg ColumnTypeChange to fullfill.
+     */
+    public static void convert(ColumnType colType, ColumnTypeChange colTypeChg) {
+        colTypeChg.changeType(colType.typeSpec().name());
+        // TODO varlen types
+    }
+
+    /**
+     * Convert ColumnTypeView to ColumnType.
+     *
+     * @param colTypeView ColumnTypeView.
+     * @return ColumnType.
+     */
+    public static ColumnType convert(ColumnTypeView colTypeView) {
+        String typeName = colTypeView.type();
+        ColumnType res = types.get(typeName);
+        if (res != null)
+            return res;
+        else
+            // TODO varlen types
+            throw new IllegalArgumentException();
+    }
+
+    /**
+     * Convert column to column change.
+     *
+     * @param col Column to convert.
+     * @param colChg Column
+     */
+    public static void convert(Column col, ColumnChange colChg) {
+        colChg.changeName(col.name());
+        colChg.changeType(colTypeInit -> convert(col.type(), colTypeInit));
+        if (col.defaultValue() != null)
+            colChg.changeDefaultValue(col.defaultValue().toString()); // TODO: specify "default type" type
+        colChg.changeNullable(col.nullable());
+    }
+
+    /**
+     * Convert column view to Column.
+     *
+     * @param colView Column view.
+     * @return Column.
+     */
+    public static Column convert(ColumnView colView) {
+        String name = colView.name();
+        ColumnType type = convert(colView.type());
+        boolean nullable = colView.nullable();
+        String defValue = colView.defaultValue();
+
+        return new ColumnImpl(name, type, nullable, defValue);
+    }
+
+    /**
+     * Convert schema table to schema table change.
+     *
+     * @param tbl Schema table to convert.
+     * @param tblChg Change to fullfill.
+     */
+    public static void convert(SchemaTable tbl, TableChange tblChg) {
+        tblChg.changeName(tbl.canonicalName());
+        tblChg.changeIndices(idxsChg -> {
+            for (TableIndex idx : tbl.indices())
+                idxsChg.create(idx.name(), idxInit -> convert(idx, idxInit));
+        });
+        tblChg.changeColumns(colsChg -> {
+            for (Column col : tbl.keyColumns())
+                colsChg.create(col.name(), colChg -> convert(col, colChg));
+            for (Column col : tbl.valueColumns())
+                colsChg.create(col.name(), colChg -> convert(col, colChg));
+        });
+    }
+
+    public static SchemaTable convert(TableConfiguration tblCfg) {
+        return convert(tblCfg.value());
+    }
+
+    /**
+     * Convert configuration to SchemaTable.
+     *
+     * @param tblView SchmTableView to convert.
+     * @return SchemaTable.
+     */
+    public static SchemaTableImpl convert(TableView tblView) {
+        String canonicalName = tblView.name();
+
+        NamedListView<? extends ColumnView> colsView = tblView.columns();
+        LinkedHashMap<String, Column> columns = new LinkedHashMap<>(colsView.size());
+        for (String key : colsView.namedListKeys()) {
+            ColumnView colView = colsView.get(key);
+            Column col = convert(colView);
+            columns.put(col.name(), col);
+        }
+
+        NamedListView<? extends TableIndexView> idxsView = tblView.indices();
+        Map<String, TableIndex> indices = new HashMap<>(idxsView.size());
+        for (String key : idxsView.namedListKeys()) {
+            TableIndexView idxView = idxsView.get(key);
+            TableIndex idx = convert(idxView);
+            indices.put(idx.name(), idx);

Review comment:
       Empty line.




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

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



[GitHub] [ignite-3] AMashenkov commented on a change in pull request #132: GG-14290: SchemaDescriptor and ConfigurationSchema converters.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #132:
URL: https://github.com/apache/ignite-3/pull/132#discussion_r635025719



##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaTableValidatorImpl.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.TableValidator;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.validation.ValidationContext;
+import org.apache.ignite.configuration.validation.ValidationIssue;
+import org.apache.ignite.configuration.validation.Validator;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.builder.SchemaTableBuilderImpl;
+import org.apache.ignite.schema.Column;
+
+import java.util.ArrayList;
+import java.util.Collection;
+
+/** SchemaTable validator implementation.  */
+public class SchemaTableValidatorImpl implements Validator<TableValidator, TableView> {
+    public final static SchemaTableValidatorImpl INSTANCE = new SchemaTableValidatorImpl();
+
+    @Override public void validate(TableValidator annotation, ValidationContext<TableView> ctx) {
+        try {
+            SchemaTableImpl tbl = SchemaConfigurationConverter.convert(ctx.getNewValue());
+            Collection<Column> allColumns = new ArrayList<>(tbl.keyColumns());
+            allColumns.addAll(tbl.valueColumns());
+            SchemaTableBuilderImpl.validateIndices(tbl.indices(), allColumns);

Review comment:
       Empty lines.

##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaTableValidatorImpl.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.TableValidator;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.validation.ValidationContext;
+import org.apache.ignite.configuration.validation.ValidationIssue;
+import org.apache.ignite.configuration.validation.Validator;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.builder.SchemaTableBuilderImpl;
+import org.apache.ignite.schema.Column;
+
+import java.util.ArrayList;
+import java.util.Collection;
+
+/** SchemaTable validator implementation.  */
+public class SchemaTableValidatorImpl implements Validator<TableValidator, TableView> {
+    public final static SchemaTableValidatorImpl INSTANCE = new SchemaTableValidatorImpl();
+
+    @Override public void validate(TableValidator annotation, ValidationContext<TableView> ctx) {

Review comment:
       Javadoc.




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

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



[GitHub] [ignite-3] AMashenkov commented on a change in pull request #132: GG-14290: SchemaDescriptor and ConfigurationSchema converters.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #132:
URL: https://github.com/apache/ignite-3/pull/132#discussion_r635023033



##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnChange;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexChange;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.schemas.table.TablesChange;
+import org.apache.ignite.configuration.tree.NamedListView;
+import org.apache.ignite.internal.schema.ColumnImpl;
+import org.apache.ignite.internal.schema.HashIndexImpl;
+import org.apache.ignite.internal.schema.PartialIndexImpl;
+import org.apache.ignite.internal.schema.PrimaryIndexImpl;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.SortedIndexColumnImpl;
+import org.apache.ignite.internal.schema.SortedIndexImpl;
+import org.apache.ignite.schema.Column;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.IndexColumn;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
+import org.apache.ignite.schema.SortedIndexColumn;
+import org.apache.ignite.schema.TableIndex;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/** Configuration to schema and vice versa converter. */
+public class SchemaConfigurationConverter {
+    /** Hash index type. */
+    private static final String HASH_TYPE = "HASH";
+
+    /** Sorted index type. */
+    private static final String SORTED_TYPE = "SORTED";
+
+    /** Partial index type. */
+    private static final String PARTIAL_TYPE = "PARTIAL";
+
+    /** Primary key index type. */
+    private static final String PK_TYPE = "PK";
+
+    /** Types map. */
+    private static Map<String, ColumnType> types = new HashMap<>();
+
+    static {
+        putType(ColumnType.INT8);
+        putType(ColumnType.INT16);
+        putType(ColumnType.INT32);
+        putType(ColumnType.INT64);
+        putType(ColumnType.UINT8);
+        putType(ColumnType.UINT16);
+        putType(ColumnType.UINT32);
+        putType(ColumnType.UINT64);
+        putType(ColumnType.FLOAT);
+        putType(ColumnType.DOUBLE);
+        putType(ColumnType.UUID);
+        putType(ColumnType.string());
+        putType(ColumnType.blobOf());
+        // TODO; handle length for some types
+        //putType(ColumnType.bitmaskOf());
+    }
+
+    private static void putType(ColumnType type) {
+        types.put(type.typeSpec().name(), type);
+    }
+
+    /**
+     * Convert SortedIndexColumn to IndexColumnChange.
+     * @param col IndexColumnChange.
+     * @param colInit IndexColumnChange to fullfill.
+     */
+    public static void convert(SortedIndexColumn col, IndexColumnChange colInit) {
+        colInit.changeName(col.name());
+        colInit.changeAsc(col.asc());
+    }
+
+    /**
+     * Convert IndexColumnView to SortedIndexColumn.
+     *
+     * @param colCfg IndexColumnView.
+     * @return SortedIndexColumn.
+     */
+    public static SortedIndexColumn convert(IndexColumnView colCfg) {
+        return new SortedIndexColumnImpl(colCfg.name(), colCfg.asc());
+    }
+
+    /**
+     * Convert TableIndex to TableIndexChange.
+     *
+     * @param idx TableIndex.
+     * @param idxChg TableIndexChange to fullfill.
+     */
+    public static void convert(TableIndex idx, TableIndexChange idxChg) {
+        idxChg.changeName(idx.name());
+        idxChg.changeType(idx.type());
+
+        if (HASH_TYPE.equals(idx.type())) {
+            HashIndex hashIdx = (HashIndex)idx;
+
+            String[] colNames = hashIdx.columns().stream().map(IndexColumn::name).toArray(String[]::new);
+            idxChg.changeColNames(colNames);
+        }
+        else if (PARTIAL_TYPE.equals(idx.type())) {
+            PartialIndex partIdx = (PartialIndex)idx;
+
+            idxChg.changeUniq(partIdx.unique());
+            idxChg.changeExpr(partIdx.expr());
+
+            idxChg.changeColumns(colsChg -> {
+                for (SortedIndexColumn col : partIdx.columns())
+                    colsChg.create(col.name(), colInit -> convert(col, colInit));
+            });
+
+        }
+        else if (SORTED_TYPE.equals(idx.type())) {
+            SortedIndex sortIdx = (SortedIndex)idx;
+            idxChg.changeUniq(sortIdx.unique());
+
+            idxChg.changeColumns(colsInit -> {
+                for (SortedIndexColumn col : sortIdx.columns())
+                    colsInit.create(col.name(), colInit -> convert(col, colInit));
+            });
+        }
+        else if (PK_TYPE.equals(idx.type())) {
+            PrimaryIndex primIdx = (PrimaryIndex)idx;
+
+            idxChg.changeColumns(colsInit -> {
+                for (SortedIndexColumn col : primIdx.columns())
+                    colsInit.create(col.name(), colInit -> convert(col, colInit));
+            });
+
+            idxChg.changeAffinityColumns(primIdx.affinityColumns().toArray(new String[primIdx.affinityColumns().size()]));
+        }
+        else throw new IllegalArgumentException("Unknown index type " + idx.type());
+    }
+
+    /**
+     * Convert TableIndexView into TableIndex.
+     *
+     * @param idxView TableIndexView.
+     * @return TableIndex.
+     */
+    public static TableIndex convert(TableIndexView idxView) {
+        String name = idxView.name();
+        String type = idxView.type();
+
+        if (type.equals("HASH")) {
+            String[] cols = idxView.colNames();
+
+            return new HashIndexImpl(name, cols);
+        }
+        else if (type.equals("SORTED") || type.equals("PARTIAL") || type.equals("PK")) {
+            if (type.equals("PARTIAL")) {
+                boolean uniq = idxView.uniq();
+                String expr = idxView.expr();
+                NamedListView<? extends IndexColumnView> colsView = idxView.columns();
+                List<SortedIndexColumn> cols = new ArrayList<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(colsView.get(key));
+                    cols.add(col);
+                }
+
+                return new PartialIndexImpl(name, cols, uniq, expr);
+            }
+            if (type.equals("SORTED")) {
+                boolean uniq = idxView.uniq();
+                List<SortedIndexColumn> cols = new ArrayList<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    cols.add(col);
+                }
+
+                return new SortedIndexImpl(name, cols, uniq);
+            }
+            if (type.equals("PK")) {
+                List<SortedIndexColumn> cols = new ArrayList<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    cols.add(col);
+                }
+                String[] affCols = idxView.colNames();
+
+                return new PrimaryIndexImpl(cols, Arrays.stream(affCols).collect(Collectors.toList()));
+            }
+
+            return null;
+        }
+        else
+            throw new IllegalArgumentException("Unknown type " + type);
+    }
+
+    /**
+     * Convert ColumnType to ColumnTypeChange.
+     *
+     * @param colType ColumnType.
+     * @param colTypeChg ColumnTypeChange to fullfill.
+     */
+    public static void convert(ColumnType colType, ColumnTypeChange colTypeChg) {
+        colTypeChg.changeType(colType.typeSpec().name());
+        // TODO varlen types
+    }
+
+    /**
+     * Convert ColumnTypeView to ColumnType.
+     *
+     * @param colTypeView ColumnTypeView.
+     * @return ColumnType.
+     */
+    public static ColumnType convert(ColumnTypeView colTypeView) {
+        String typeName = colTypeView.type();
+        ColumnType res = types.get(typeName);
+        if (res != null)
+            return res;
+        else
+            // TODO varlen types
+            throw new IllegalArgumentException();
+    }
+
+    /**
+     * Convert column to column change.
+     *
+     * @param col Column to convert.
+     * @param colChg Column
+     */
+    public static void convert(Column col, ColumnChange colChg) {
+        colChg.changeName(col.name());
+        colChg.changeType(colTypeInit -> convert(col.type(), colTypeInit));
+        if (col.defaultValue() != null)
+            colChg.changeDefaultValue(col.defaultValue().toString()); // TODO: specify "default type" type
+        colChg.changeNullable(col.nullable());
+    }
+
+    /**
+     * Convert column view to Column.
+     *
+     * @param colView Column view.
+     * @return Column.
+     */
+    public static Column convert(ColumnView colView) {
+        String name = colView.name();
+        ColumnType type = convert(colView.type());
+        boolean nullable = colView.nullable();
+        String defValue = colView.defaultValue();
+
+        return new ColumnImpl(name, type, nullable, defValue);
+    }
+
+    /**
+     * Convert schema table to schema table change.
+     *
+     * @param tbl Schema table to convert.
+     * @param tblChg Change to fullfill.
+     */
+    public static void convert(SchemaTable tbl, TableChange tblChg) {
+        tblChg.changeName(tbl.canonicalName());
+        tblChg.changeIndices(idxsChg -> {
+            for (TableIndex idx : tbl.indices())
+                idxsChg.create(idx.name(), idxInit -> convert(idx, idxInit));
+        });
+        tblChg.changeColumns(colsChg -> {
+            for (Column col : tbl.keyColumns())
+                colsChg.create(col.name(), colChg -> convert(col, colChg));
+            for (Column col : tbl.valueColumns())
+                colsChg.create(col.name(), colChg -> convert(col, colChg));
+        });
+    }
+
+    public static SchemaTable convert(TableConfiguration tblCfg) {
+        return convert(tblCfg.value());
+    }
+
+    /**
+     * Convert configuration to SchemaTable.
+     *
+     * @param tblView SchmTableView to convert.
+     * @return SchemaTable.
+     */
+    public static SchemaTableImpl convert(TableView tblView) {
+        String canonicalName = tblView.name();
+
+        NamedListView<? extends ColumnView> colsView = tblView.columns();
+        LinkedHashMap<String, Column> columns = new LinkedHashMap<>(colsView.size());
+        for (String key : colsView.namedListKeys()) {
+            ColumnView colView = colsView.get(key);
+            Column col = convert(colView);
+            columns.put(col.name(), col);
+        }
+
+        NamedListView<? extends TableIndexView> idxsView = tblView.indices();
+        Map<String, TableIndex> indices = new HashMap<>(idxsView.size());
+        for (String key : idxsView.namedListKeys()) {
+            TableIndexView idxView = idxsView.get(key);
+            TableIndex idx = convert(idxView);
+            indices.put(idx.name(), idx);
+        }
+
+        // TUDO: schema name

Review comment:
       TODO with no ticket.




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

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



[GitHub] [ignite-3] AMashenkov commented on a change in pull request #132: GG-14290: SchemaDescriptor and ConfigurationSchema converters.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #132:
URL: https://github.com/apache/ignite-3/pull/132#discussion_r635036991



##########
File path: modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/TablesConfigurationSchema.java
##########
@@ -25,8 +25,8 @@
 @SuppressWarnings("PMD.UnusedPrivateField")
 @ConfigurationRoot(rootName = "table", type = ConfigurationType.DISTRIBUTED)
 public class TablesConfigurationSchema {
-
     /** List of configured tables. */
     @NamedConfigValue
+    //@TableValidator

Review comment:
       Uncomment ?




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

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



[GitHub] [ignite-3] AMashenkov commented on a change in pull request #132: GG-14290: SchemaDescriptor and ConfigurationSchema converters.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #132:
URL: https://github.com/apache/ignite-3/pull/132#discussion_r635043779



##########
File path: modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/TableValidator.java
##########
@@ -0,0 +1,36 @@
+/*
+ * 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.configuration.schemas.table;
+
+import java.lang.annotation.Retention;
+import java.lang.annotation.Target;
+
+import static java.lang.annotation.ElementType.ANNOTATION_TYPE;
+import static java.lang.annotation.ElementType.CONSTRUCTOR;
+import static java.lang.annotation.ElementType.FIELD;
+import static java.lang.annotation.ElementType.METHOD;
+import static java.lang.annotation.ElementType.PARAMETER;
+import static java.lang.annotation.ElementType.TYPE_USE;
+import static java.lang.annotation.RetentionPolicy.RUNTIME;
+
+/** Annotation to validate whole table configuration. */

Review comment:
       Please, describe annotation effect.




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

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



[GitHub] [ignite-3] AMashenkov commented on a change in pull request #132: IGNITE-14290: Schema configuration API implementation.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #132:
URL: https://github.com/apache/ignite-3/pull/132#discussion_r637737700



##########
File path: modules/schema/src/test/java/org/apache/ignite/internal/schema/builder/HashIndexBuilderTest.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.ignite.internal.schema.builder;
+
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.SchemaBuilders;
+import org.apache.ignite.schema.builder.HashIndexBuilder;
+import org.junit.jupiter.api.Test;
+
+import java.util.Collections;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+/** Tests for hash index builder. */
+public class HashIndexBuilderTest {
+    /** Full featured HashIndex. */
+    public static final HashIndex IDX;
+
+    static {
+        HashIndexBuilder builder = SchemaBuilders.hashIndex("testHI")
+            .withColumns("A", "B", "C")
+            .withHints(Collections.singletonMap("param","value"));
+        IDX = builder.build();

Review comment:
       ```suggestion
   
           IDX = builder.build();
   ```




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

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



[GitHub] [ignite-3] AMashenkov commented on a change in pull request #132: IGNITE-14290: Schema configuration API implementation.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #132:
URL: https://github.com/apache/ignite-3/pull/132#discussion_r637738713



##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
##########
@@ -0,0 +1,489 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnChange;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexChange;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.schemas.table.TablesChange;
+import org.apache.ignite.configuration.tree.NamedListView;
+import org.apache.ignite.internal.schema.ColumnImpl;
+import org.apache.ignite.internal.schema.HashIndexImpl;
+import org.apache.ignite.internal.schema.PartialIndexImpl;
+import org.apache.ignite.internal.schema.PrimaryIndexImpl;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.SortedIndexColumnImpl;
+import org.apache.ignite.internal.schema.SortedIndexImpl;
+import org.apache.ignite.schema.Column;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.IndexColumn;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
+import org.apache.ignite.schema.SortedIndexColumn;
+import org.apache.ignite.schema.TableIndex;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+/**
+ * Configuration to schema and vice versa converter.
+ */
+public class SchemaConfigurationConverter {
+    /** Hash index type. */
+    private static final String HASH_TYPE = "HASH";
+
+    /** Sorted index type. */
+    private static final String SORTED_TYPE = "SORTED";
+
+    /** Partial index type. */
+    private static final String PARTIAL_TYPE = "PARTIAL";
+
+    /** Primary key index type. */
+    private static final String PK_TYPE = "PK";
+
+    /** Types map. */
+    private static Map<String, ColumnType> types = new HashMap<>();
+
+    static {
+        putType(ColumnType.INT8);
+        putType(ColumnType.INT16);
+        putType(ColumnType.INT32);
+        putType(ColumnType.INT64);
+        putType(ColumnType.UINT8);
+        putType(ColumnType.UINT16);
+        putType(ColumnType.UINT32);
+        putType(ColumnType.UINT64);
+        putType(ColumnType.FLOAT);
+        putType(ColumnType.DOUBLE);
+        putType(ColumnType.UUID);
+    }
+
+    /** */
+    private static void putType(ColumnType type) {
+        types.put(type.typeSpec().name(), type);
+    }
+
+    /**
+     * Convert SortedIndexColumn to IndexColumnChange.
+     * @param col IndexColumnChange.
+     * @param colInit IndexColumnChange to fullfill.
+     */
+    public static void convert(SortedIndexColumn col, IndexColumnChange colInit) {
+        colInit.changeName(col.name());
+        colInit.changeAsc(col.asc());
+    }
+
+    /**
+     * Convert IndexColumnView to SortedIndexColumn.
+     *
+     * @param colCfg IndexColumnView.
+     * @return SortedIndexColumn.
+     */
+    public static SortedIndexColumn convert(IndexColumnView colCfg) {
+        return new SortedIndexColumnImpl(colCfg.name(), colCfg.asc());
+    }
+
+    /**
+     * Convert TableIndex to TableIndexChange.
+     *
+     * @param idx TableIndex.
+     * @param idxChg TableIndexChange to fullfill.
+     */
+    public static void convert(TableIndex idx, TableIndexChange idxChg) {
+        idxChg.changeName(idx.name());
+        idxChg.changeType(idx.type());
+
+        switch (idx.type()) {
+            case HASH_TYPE:
+                HashIndex hashIdx = (HashIndex)idx;
+
+                String[] colNames = hashIdx.columns().stream().map(IndexColumn::name).toArray(String[]::new);
+
+                idxChg.changeColNames(colNames);
+
+                break;
+
+            case PARTIAL_TYPE:
+                PartialIndex partIdx = (PartialIndex)idx;
+
+                idxChg.changeUniq(partIdx.unique());
+                idxChg.changeExpr(partIdx.expr());
+
+                idxChg.changeColumns(colsChg -> {
+                    int colIdx = 0;
+
+                    for (SortedIndexColumn col : partIdx.columns())
+                        colsChg.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+
+                break;
+
+            case SORTED_TYPE:
+                SortedIndex sortIdx = (SortedIndex)idx;
+                idxChg.changeUniq(sortIdx.unique());
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+
+                    for (SortedIndexColumn col : sortIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+
+                break;
+
+            case PK_TYPE:
+                PrimaryIndex primIdx = (PrimaryIndex)idx;
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+
+                    for (SortedIndexColumn col : primIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+
+                idxChg.changeAffinityColumns(primIdx.affinityColumns().toArray(
+                    new String[primIdx.affinityColumns().size()]));
+
+                break;
+
+            default:
+                throw new IllegalArgumentException("Unknown index type " + idx.type());
+        }
+    }
+
+    /**
+     * Convert TableIndexView into TableIndex.
+     *
+     * @param idxView TableIndexView.
+     * @return TableIndex.
+     */
+    public static TableIndex convert(TableIndexView idxView) {
+        String name = idxView.name();
+        String type = idxView.type();
+
+        switch (type) {
+            case "HASH":
+                String[] hashCols = idxView.colNames();
+
+                return new HashIndexImpl(name, hashCols);
+
+            case "SORTED":
+                boolean sortedUniq = idxView.uniq();
+
+                SortedMap<Integer, SortedIndexColumn> sortedCols = new TreeMap();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    sortedCols.put(Integer.valueOf(key), col);

Review comment:
       ```suggestion
                       SortedIndexColumn col = convert(idxView.columns().get(key));
                       
                       sortedCols.put(Integer.valueOf(key), col);
   ```

##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
##########
@@ -0,0 +1,489 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnChange;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexChange;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.schemas.table.TablesChange;
+import org.apache.ignite.configuration.tree.NamedListView;
+import org.apache.ignite.internal.schema.ColumnImpl;
+import org.apache.ignite.internal.schema.HashIndexImpl;
+import org.apache.ignite.internal.schema.PartialIndexImpl;
+import org.apache.ignite.internal.schema.PrimaryIndexImpl;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.SortedIndexColumnImpl;
+import org.apache.ignite.internal.schema.SortedIndexImpl;
+import org.apache.ignite.schema.Column;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.IndexColumn;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
+import org.apache.ignite.schema.SortedIndexColumn;
+import org.apache.ignite.schema.TableIndex;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+/**
+ * Configuration to schema and vice versa converter.
+ */
+public class SchemaConfigurationConverter {
+    /** Hash index type. */
+    private static final String HASH_TYPE = "HASH";
+
+    /** Sorted index type. */
+    private static final String SORTED_TYPE = "SORTED";
+
+    /** Partial index type. */
+    private static final String PARTIAL_TYPE = "PARTIAL";
+
+    /** Primary key index type. */
+    private static final String PK_TYPE = "PK";
+
+    /** Types map. */
+    private static Map<String, ColumnType> types = new HashMap<>();
+
+    static {
+        putType(ColumnType.INT8);
+        putType(ColumnType.INT16);
+        putType(ColumnType.INT32);
+        putType(ColumnType.INT64);
+        putType(ColumnType.UINT8);
+        putType(ColumnType.UINT16);
+        putType(ColumnType.UINT32);
+        putType(ColumnType.UINT64);
+        putType(ColumnType.FLOAT);
+        putType(ColumnType.DOUBLE);
+        putType(ColumnType.UUID);
+    }
+
+    /** */
+    private static void putType(ColumnType type) {
+        types.put(type.typeSpec().name(), type);
+    }
+
+    /**
+     * Convert SortedIndexColumn to IndexColumnChange.
+     * @param col IndexColumnChange.
+     * @param colInit IndexColumnChange to fullfill.
+     */
+    public static void convert(SortedIndexColumn col, IndexColumnChange colInit) {
+        colInit.changeName(col.name());
+        colInit.changeAsc(col.asc());
+    }
+
+    /**
+     * Convert IndexColumnView to SortedIndexColumn.
+     *
+     * @param colCfg IndexColumnView.
+     * @return SortedIndexColumn.
+     */
+    public static SortedIndexColumn convert(IndexColumnView colCfg) {
+        return new SortedIndexColumnImpl(colCfg.name(), colCfg.asc());
+    }
+
+    /**
+     * Convert TableIndex to TableIndexChange.
+     *
+     * @param idx TableIndex.
+     * @param idxChg TableIndexChange to fullfill.
+     */
+    public static void convert(TableIndex idx, TableIndexChange idxChg) {
+        idxChg.changeName(idx.name());
+        idxChg.changeType(idx.type());
+
+        switch (idx.type()) {
+            case HASH_TYPE:
+                HashIndex hashIdx = (HashIndex)idx;
+
+                String[] colNames = hashIdx.columns().stream().map(IndexColumn::name).toArray(String[]::new);
+
+                idxChg.changeColNames(colNames);
+
+                break;
+
+            case PARTIAL_TYPE:
+                PartialIndex partIdx = (PartialIndex)idx;
+
+                idxChg.changeUniq(partIdx.unique());
+                idxChg.changeExpr(partIdx.expr());
+
+                idxChg.changeColumns(colsChg -> {
+                    int colIdx = 0;
+
+                    for (SortedIndexColumn col : partIdx.columns())
+                        colsChg.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+
+                break;
+
+            case SORTED_TYPE:
+                SortedIndex sortIdx = (SortedIndex)idx;
+                idxChg.changeUniq(sortIdx.unique());
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+
+                    for (SortedIndexColumn col : sortIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+
+                break;
+
+            case PK_TYPE:
+                PrimaryIndex primIdx = (PrimaryIndex)idx;
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+
+                    for (SortedIndexColumn col : primIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+
+                idxChg.changeAffinityColumns(primIdx.affinityColumns().toArray(
+                    new String[primIdx.affinityColumns().size()]));
+
+                break;
+
+            default:
+                throw new IllegalArgumentException("Unknown index type " + idx.type());
+        }
+    }
+
+    /**
+     * Convert TableIndexView into TableIndex.
+     *
+     * @param idxView TableIndexView.
+     * @return TableIndex.
+     */
+    public static TableIndex convert(TableIndexView idxView) {
+        String name = idxView.name();
+        String type = idxView.type();
+
+        switch (type) {
+            case "HASH":
+                String[] hashCols = idxView.colNames();
+
+                return new HashIndexImpl(name, hashCols);
+
+            case "SORTED":
+                boolean sortedUniq = idxView.uniq();
+
+                SortedMap<Integer, SortedIndexColumn> sortedCols = new TreeMap();
+                for (String key : idxView.columns().namedListKeys()) {

Review comment:
       ```suggestion
                   SortedMap<Integer, SortedIndexColumn> sortedCols = new TreeMap();
                   
                   for (String key : idxView.columns().namedListKeys()) {
   ```

##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
##########
@@ -0,0 +1,489 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnChange;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexChange;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.schemas.table.TablesChange;
+import org.apache.ignite.configuration.tree.NamedListView;
+import org.apache.ignite.internal.schema.ColumnImpl;
+import org.apache.ignite.internal.schema.HashIndexImpl;
+import org.apache.ignite.internal.schema.PartialIndexImpl;
+import org.apache.ignite.internal.schema.PrimaryIndexImpl;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.SortedIndexColumnImpl;
+import org.apache.ignite.internal.schema.SortedIndexImpl;
+import org.apache.ignite.schema.Column;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.IndexColumn;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
+import org.apache.ignite.schema.SortedIndexColumn;
+import org.apache.ignite.schema.TableIndex;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+/**
+ * Configuration to schema and vice versa converter.
+ */
+public class SchemaConfigurationConverter {
+    /** Hash index type. */
+    private static final String HASH_TYPE = "HASH";
+
+    /** Sorted index type. */
+    private static final String SORTED_TYPE = "SORTED";
+
+    /** Partial index type. */
+    private static final String PARTIAL_TYPE = "PARTIAL";
+
+    /** Primary key index type. */
+    private static final String PK_TYPE = "PK";
+
+    /** Types map. */
+    private static Map<String, ColumnType> types = new HashMap<>();
+
+    static {
+        putType(ColumnType.INT8);
+        putType(ColumnType.INT16);
+        putType(ColumnType.INT32);
+        putType(ColumnType.INT64);
+        putType(ColumnType.UINT8);
+        putType(ColumnType.UINT16);
+        putType(ColumnType.UINT32);
+        putType(ColumnType.UINT64);
+        putType(ColumnType.FLOAT);
+        putType(ColumnType.DOUBLE);
+        putType(ColumnType.UUID);
+    }
+
+    /** */
+    private static void putType(ColumnType type) {
+        types.put(type.typeSpec().name(), type);
+    }
+
+    /**
+     * Convert SortedIndexColumn to IndexColumnChange.
+     * @param col IndexColumnChange.
+     * @param colInit IndexColumnChange to fullfill.
+     */
+    public static void convert(SortedIndexColumn col, IndexColumnChange colInit) {
+        colInit.changeName(col.name());
+        colInit.changeAsc(col.asc());
+    }
+
+    /**
+     * Convert IndexColumnView to SortedIndexColumn.
+     *
+     * @param colCfg IndexColumnView.
+     * @return SortedIndexColumn.
+     */
+    public static SortedIndexColumn convert(IndexColumnView colCfg) {
+        return new SortedIndexColumnImpl(colCfg.name(), colCfg.asc());
+    }
+
+    /**
+     * Convert TableIndex to TableIndexChange.
+     *
+     * @param idx TableIndex.
+     * @param idxChg TableIndexChange to fullfill.
+     */
+    public static void convert(TableIndex idx, TableIndexChange idxChg) {
+        idxChg.changeName(idx.name());
+        idxChg.changeType(idx.type());
+
+        switch (idx.type()) {
+            case HASH_TYPE:
+                HashIndex hashIdx = (HashIndex)idx;
+
+                String[] colNames = hashIdx.columns().stream().map(IndexColumn::name).toArray(String[]::new);
+
+                idxChg.changeColNames(colNames);
+
+                break;
+
+            case PARTIAL_TYPE:
+                PartialIndex partIdx = (PartialIndex)idx;
+
+                idxChg.changeUniq(partIdx.unique());
+                idxChg.changeExpr(partIdx.expr());
+
+                idxChg.changeColumns(colsChg -> {
+                    int colIdx = 0;
+
+                    for (SortedIndexColumn col : partIdx.columns())
+                        colsChg.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+
+                break;
+
+            case SORTED_TYPE:
+                SortedIndex sortIdx = (SortedIndex)idx;
+                idxChg.changeUniq(sortIdx.unique());
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+
+                    for (SortedIndexColumn col : sortIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+
+                break;
+
+            case PK_TYPE:
+                PrimaryIndex primIdx = (PrimaryIndex)idx;
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+
+                    for (SortedIndexColumn col : primIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+
+                idxChg.changeAffinityColumns(primIdx.affinityColumns().toArray(
+                    new String[primIdx.affinityColumns().size()]));
+
+                break;
+
+            default:
+                throw new IllegalArgumentException("Unknown index type " + idx.type());
+        }
+    }
+
+    /**
+     * Convert TableIndexView into TableIndex.
+     *
+     * @param idxView TableIndexView.
+     * @return TableIndex.
+     */
+    public static TableIndex convert(TableIndexView idxView) {
+        String name = idxView.name();
+        String type = idxView.type();
+
+        switch (type) {
+            case "HASH":
+                String[] hashCols = idxView.colNames();
+
+                return new HashIndexImpl(name, hashCols);
+
+            case "SORTED":
+                boolean sortedUniq = idxView.uniq();
+
+                SortedMap<Integer, SortedIndexColumn> sortedCols = new TreeMap();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    sortedCols.put(Integer.valueOf(key), col);
+                }
+
+                return new SortedIndexImpl(name, new ArrayList<>(sortedCols.values()), sortedUniq);
+
+            case "PARTIAL":
+                boolean partialUniq = idxView.uniq();
+                String expr = idxView.expr();
+
+                NamedListView<? extends IndexColumnView> colsView = idxView.columns();
+                SortedMap<Integer, SortedIndexColumn> partialCols = new TreeMap<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(colsView.get(key));
+                    partialCols.put(Integer.valueOf(key), col);

Review comment:
       ```suggestion
                   NamedListView<? extends IndexColumnView> colsView = idxView.columns();
                   SortedMap<Integer, SortedIndexColumn> partialCols = new TreeMap<>();
                   
                   for (String key : idxView.columns().namedListKeys()) {
                       SortedIndexColumn col = convert(colsView.get(key));
                       
                       partialCols.put(Integer.valueOf(key), col);
   ```

##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
##########
@@ -0,0 +1,489 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnChange;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexChange;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.schemas.table.TablesChange;
+import org.apache.ignite.configuration.tree.NamedListView;
+import org.apache.ignite.internal.schema.ColumnImpl;
+import org.apache.ignite.internal.schema.HashIndexImpl;
+import org.apache.ignite.internal.schema.PartialIndexImpl;
+import org.apache.ignite.internal.schema.PrimaryIndexImpl;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.SortedIndexColumnImpl;
+import org.apache.ignite.internal.schema.SortedIndexImpl;
+import org.apache.ignite.schema.Column;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.IndexColumn;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
+import org.apache.ignite.schema.SortedIndexColumn;
+import org.apache.ignite.schema.TableIndex;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+/**
+ * Configuration to schema and vice versa converter.
+ */
+public class SchemaConfigurationConverter {
+    /** Hash index type. */
+    private static final String HASH_TYPE = "HASH";
+
+    /** Sorted index type. */
+    private static final String SORTED_TYPE = "SORTED";
+
+    /** Partial index type. */
+    private static final String PARTIAL_TYPE = "PARTIAL";
+
+    /** Primary key index type. */
+    private static final String PK_TYPE = "PK";
+
+    /** Types map. */
+    private static Map<String, ColumnType> types = new HashMap<>();
+
+    static {
+        putType(ColumnType.INT8);
+        putType(ColumnType.INT16);
+        putType(ColumnType.INT32);
+        putType(ColumnType.INT64);
+        putType(ColumnType.UINT8);
+        putType(ColumnType.UINT16);
+        putType(ColumnType.UINT32);
+        putType(ColumnType.UINT64);
+        putType(ColumnType.FLOAT);
+        putType(ColumnType.DOUBLE);
+        putType(ColumnType.UUID);
+    }
+
+    /** */
+    private static void putType(ColumnType type) {
+        types.put(type.typeSpec().name(), type);
+    }
+
+    /**
+     * Convert SortedIndexColumn to IndexColumnChange.
+     * @param col IndexColumnChange.
+     * @param colInit IndexColumnChange to fullfill.
+     */
+    public static void convert(SortedIndexColumn col, IndexColumnChange colInit) {
+        colInit.changeName(col.name());
+        colInit.changeAsc(col.asc());
+    }
+
+    /**
+     * Convert IndexColumnView to SortedIndexColumn.
+     *
+     * @param colCfg IndexColumnView.
+     * @return SortedIndexColumn.
+     */
+    public static SortedIndexColumn convert(IndexColumnView colCfg) {
+        return new SortedIndexColumnImpl(colCfg.name(), colCfg.asc());
+    }
+
+    /**
+     * Convert TableIndex to TableIndexChange.
+     *
+     * @param idx TableIndex.
+     * @param idxChg TableIndexChange to fullfill.
+     */
+    public static void convert(TableIndex idx, TableIndexChange idxChg) {
+        idxChg.changeName(idx.name());
+        idxChg.changeType(idx.type());
+
+        switch (idx.type()) {
+            case HASH_TYPE:
+                HashIndex hashIdx = (HashIndex)idx;
+
+                String[] colNames = hashIdx.columns().stream().map(IndexColumn::name).toArray(String[]::new);
+
+                idxChg.changeColNames(colNames);
+
+                break;
+
+            case PARTIAL_TYPE:
+                PartialIndex partIdx = (PartialIndex)idx;
+
+                idxChg.changeUniq(partIdx.unique());
+                idxChg.changeExpr(partIdx.expr());
+
+                idxChg.changeColumns(colsChg -> {
+                    int colIdx = 0;
+
+                    for (SortedIndexColumn col : partIdx.columns())
+                        colsChg.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+
+                break;
+
+            case SORTED_TYPE:
+                SortedIndex sortIdx = (SortedIndex)idx;
+                idxChg.changeUniq(sortIdx.unique());
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+
+                    for (SortedIndexColumn col : sortIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+
+                break;
+
+            case PK_TYPE:
+                PrimaryIndex primIdx = (PrimaryIndex)idx;
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+
+                    for (SortedIndexColumn col : primIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+
+                idxChg.changeAffinityColumns(primIdx.affinityColumns().toArray(
+                    new String[primIdx.affinityColumns().size()]));
+
+                break;
+
+            default:
+                throw new IllegalArgumentException("Unknown index type " + idx.type());
+        }
+    }
+
+    /**
+     * Convert TableIndexView into TableIndex.
+     *
+     * @param idxView TableIndexView.
+     * @return TableIndex.
+     */
+    public static TableIndex convert(TableIndexView idxView) {
+        String name = idxView.name();
+        String type = idxView.type();
+
+        switch (type) {
+            case "HASH":
+                String[] hashCols = idxView.colNames();
+
+                return new HashIndexImpl(name, hashCols);
+
+            case "SORTED":
+                boolean sortedUniq = idxView.uniq();
+
+                SortedMap<Integer, SortedIndexColumn> sortedCols = new TreeMap();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    sortedCols.put(Integer.valueOf(key), col);
+                }
+
+                return new SortedIndexImpl(name, new ArrayList<>(sortedCols.values()), sortedUniq);
+
+            case "PARTIAL":
+                boolean partialUniq = idxView.uniq();
+                String expr = idxView.expr();
+
+                NamedListView<? extends IndexColumnView> colsView = idxView.columns();
+                SortedMap<Integer, SortedIndexColumn> partialCols = new TreeMap<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(colsView.get(key));
+                    partialCols.put(Integer.valueOf(key), col);
+                }
+
+                return new PartialIndexImpl(name, new ArrayList<>(partialCols.values()), partialUniq, expr);
+
+            case "PK":
+                SortedMap<Integer, SortedIndexColumn> cols = new TreeMap<>();
+                for (String key : idxView.columns().namedListKeys()) {

Review comment:
       ```suggestion
                   SortedMap<Integer, SortedIndexColumn> cols = new TreeMap<>();
                   
                   for (String key : idxView.columns().namedListKeys()) {
   ```

##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
##########
@@ -0,0 +1,489 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnChange;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexChange;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.schemas.table.TablesChange;
+import org.apache.ignite.configuration.tree.NamedListView;
+import org.apache.ignite.internal.schema.ColumnImpl;
+import org.apache.ignite.internal.schema.HashIndexImpl;
+import org.apache.ignite.internal.schema.PartialIndexImpl;
+import org.apache.ignite.internal.schema.PrimaryIndexImpl;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.SortedIndexColumnImpl;
+import org.apache.ignite.internal.schema.SortedIndexImpl;
+import org.apache.ignite.schema.Column;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.IndexColumn;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
+import org.apache.ignite.schema.SortedIndexColumn;
+import org.apache.ignite.schema.TableIndex;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+/**
+ * Configuration to schema and vice versa converter.
+ */
+public class SchemaConfigurationConverter {
+    /** Hash index type. */
+    private static final String HASH_TYPE = "HASH";
+
+    /** Sorted index type. */
+    private static final String SORTED_TYPE = "SORTED";
+
+    /** Partial index type. */
+    private static final String PARTIAL_TYPE = "PARTIAL";
+
+    /** Primary key index type. */
+    private static final String PK_TYPE = "PK";
+
+    /** Types map. */
+    private static Map<String, ColumnType> types = new HashMap<>();
+
+    static {
+        putType(ColumnType.INT8);
+        putType(ColumnType.INT16);
+        putType(ColumnType.INT32);
+        putType(ColumnType.INT64);
+        putType(ColumnType.UINT8);
+        putType(ColumnType.UINT16);
+        putType(ColumnType.UINT32);
+        putType(ColumnType.UINT64);
+        putType(ColumnType.FLOAT);
+        putType(ColumnType.DOUBLE);
+        putType(ColumnType.UUID);
+    }
+
+    /** */
+    private static void putType(ColumnType type) {
+        types.put(type.typeSpec().name(), type);
+    }
+
+    /**
+     * Convert SortedIndexColumn to IndexColumnChange.
+     * @param col IndexColumnChange.
+     * @param colInit IndexColumnChange to fullfill.
+     */
+    public static void convert(SortedIndexColumn col, IndexColumnChange colInit) {
+        colInit.changeName(col.name());
+        colInit.changeAsc(col.asc());
+    }
+
+    /**
+     * Convert IndexColumnView to SortedIndexColumn.
+     *
+     * @param colCfg IndexColumnView.
+     * @return SortedIndexColumn.
+     */
+    public static SortedIndexColumn convert(IndexColumnView colCfg) {
+        return new SortedIndexColumnImpl(colCfg.name(), colCfg.asc());
+    }
+
+    /**
+     * Convert TableIndex to TableIndexChange.
+     *
+     * @param idx TableIndex.
+     * @param idxChg TableIndexChange to fullfill.
+     */
+    public static void convert(TableIndex idx, TableIndexChange idxChg) {
+        idxChg.changeName(idx.name());
+        idxChg.changeType(idx.type());
+
+        switch (idx.type()) {
+            case HASH_TYPE:
+                HashIndex hashIdx = (HashIndex)idx;
+
+                String[] colNames = hashIdx.columns().stream().map(IndexColumn::name).toArray(String[]::new);
+
+                idxChg.changeColNames(colNames);
+
+                break;
+
+            case PARTIAL_TYPE:
+                PartialIndex partIdx = (PartialIndex)idx;
+
+                idxChg.changeUniq(partIdx.unique());
+                idxChg.changeExpr(partIdx.expr());
+
+                idxChg.changeColumns(colsChg -> {
+                    int colIdx = 0;
+
+                    for (SortedIndexColumn col : partIdx.columns())
+                        colsChg.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+
+                break;
+
+            case SORTED_TYPE:
+                SortedIndex sortIdx = (SortedIndex)idx;
+                idxChg.changeUniq(sortIdx.unique());
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+
+                    for (SortedIndexColumn col : sortIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+
+                break;
+
+            case PK_TYPE:
+                PrimaryIndex primIdx = (PrimaryIndex)idx;
+
+                idxChg.changeColumns(colsInit -> {
+                    int colIdx = 0;
+
+                    for (SortedIndexColumn col : primIdx.columns())
+                        colsInit.create(String.valueOf(colIdx++), colInit -> convert(col, colInit));
+                });
+
+                idxChg.changeAffinityColumns(primIdx.affinityColumns().toArray(
+                    new String[primIdx.affinityColumns().size()]));
+
+                break;
+
+            default:
+                throw new IllegalArgumentException("Unknown index type " + idx.type());
+        }
+    }
+
+    /**
+     * Convert TableIndexView into TableIndex.
+     *
+     * @param idxView TableIndexView.
+     * @return TableIndex.
+     */
+    public static TableIndex convert(TableIndexView idxView) {
+        String name = idxView.name();
+        String type = idxView.type();
+
+        switch (type) {
+            case "HASH":
+                String[] hashCols = idxView.colNames();
+
+                return new HashIndexImpl(name, hashCols);
+
+            case "SORTED":
+                boolean sortedUniq = idxView.uniq();
+
+                SortedMap<Integer, SortedIndexColumn> sortedCols = new TreeMap();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    sortedCols.put(Integer.valueOf(key), col);
+                }
+
+                return new SortedIndexImpl(name, new ArrayList<>(sortedCols.values()), sortedUniq);
+
+            case "PARTIAL":
+                boolean partialUniq = idxView.uniq();
+                String expr = idxView.expr();
+
+                NamedListView<? extends IndexColumnView> colsView = idxView.columns();
+                SortedMap<Integer, SortedIndexColumn> partialCols = new TreeMap<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(colsView.get(key));
+                    partialCols.put(Integer.valueOf(key), col);
+                }
+
+                return new PartialIndexImpl(name, new ArrayList<>(partialCols.values()), partialUniq, expr);
+
+            case "PK":
+                SortedMap<Integer, SortedIndexColumn> cols = new TreeMap<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    cols.put(Integer.valueOf(key), col);
+                }

Review comment:
       ```suggestion
                       SortedIndexColumn col = convert(idxView.columns().get(key));
                       
                       cols.put(Integer.valueOf(key), col);
                   }
   ```

##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaTableValidatorImpl.java
##########
@@ -0,0 +1,64 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.TableValidator;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.tree.NamedListView;
+import org.apache.ignite.configuration.validation.ValidationContext;
+import org.apache.ignite.configuration.validation.ValidationIssue;
+import org.apache.ignite.configuration.validation.Validator;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.builder.SchemaTableBuilderImpl;
+import org.apache.ignite.schema.Column;
+
+import java.util.ArrayList;
+import java.util.Collection;
+
+/**
+ * SchemaTable validator implementation.
+ */
+public class SchemaTableValidatorImpl implements Validator<TableValidator, NamedListView<TableView>> {
+    /** Static instance. */
+    public static final SchemaTableValidatorImpl INSTANCE = new SchemaTableValidatorImpl();
+
+    /** {@inheritDoc} */
+    @Override public void validate(TableValidator annotation, ValidationContext<NamedListView<TableView>> ctx) {
+        NamedListView<TableView> list = ctx.getNewValue();
+        for (String key : list.namedListKeys()) {
+            TableView view = list.get(key);
+            try {

Review comment:
       ```suggestion
           NamedListView<TableView> list = ctx.getNewValue();
           
           for (String key : list.namedListKeys()) {
               TableView view = list.get(key);
               
               try {
   ```




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

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



[GitHub] [ignite-3] AMashenkov commented on a change in pull request #132: GG-14290: SchemaDescriptor and ConfigurationSchema converters.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #132:
URL: https://github.com/apache/ignite-3/pull/132#discussion_r635023355



##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaDescriptorConverter.java
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.internal.schema.Bitmask;
+import org.apache.ignite.internal.schema.Column;
+import org.apache.ignite.internal.schema.InvalidTypeException;
+import org.apache.ignite.internal.schema.NativeType;
+import org.apache.ignite.internal.schema.NativeTypeSpec;
+import org.apache.ignite.internal.schema.NumericNativeType;
+import org.apache.ignite.internal.schema.SchemaDescriptor;
+
+import org.apache.ignite.internal.schema.VarlenNativeType;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.SchemaTable;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import static org.apache.ignite.internal.schema.NativeType.BYTE;
+import static org.apache.ignite.internal.schema.NativeType.DOUBLE;
+import static org.apache.ignite.internal.schema.NativeType.FLOAT;
+import static org.apache.ignite.internal.schema.NativeType.INTEGER;
+import static org.apache.ignite.internal.schema.NativeType.LONG;
+import static org.apache.ignite.internal.schema.NativeType.SHORT;
+import static org.apache.ignite.internal.schema.NativeType.UUID;
+
+/** Build SchemaDescriptor from SchemaTable internal configuration. */
+public class SchemaDescriptorConverter {
+    /**
+     * Convert ColumnType to NativeType.
+     *
+     * @param colType ColumnType.
+     * @return NativeType.
+     */
+    private static NativeType convert(ColumnType colType) {
+        assert colType != null;
+
+        ColumnType.ColumnTypeSpec type = colType.typeSpec();
+        switch (type) {
+            case INT8:
+                return BYTE;
+
+            case INT16:
+                return SHORT;
+
+            case INT32:
+                return INTEGER;
+
+            case INT64:
+                return LONG;
+
+            case UINT8:
+            case UINT16:
+            case UINT32:
+            case UINT64:
+                throw new UnsupportedOperationException("Unsigned types are not supported yet.");
+
+            case FLOAT:
+                return FLOAT;
+
+            case DOUBLE:
+                return DOUBLE;
+
+            case DECIMAL:
+                ColumnType.NumericColumnType numType = (ColumnType.NumericColumnType)colType;
+                return new NumericNativeType(numType.precision(), numType.scale());
+
+            case UUID:
+                return UUID;
+
+            case BITMASK:
+                return new Bitmask(((ColumnType.VarLenColumnType)colType).length());
+
+            case STRING:
+                return new VarlenNativeType(NativeTypeSpec.STRING, ((ColumnType.VarLenColumnType)colType).length());
+
+            case BLOB:
+                return new VarlenNativeType(NativeTypeSpec.BYTES, ((ColumnType.VarLenColumnType)colType).length());
+
+                default:

Review comment:
       ```suggestion
               default:
   ```




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

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



[GitHub] [ignite-3] AMashenkov commented on a change in pull request #132: GG-14290: SchemaDescriptor and ConfigurationSchema converters.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #132:
URL: https://github.com/apache/ignite-3/pull/132#discussion_r635034814



##########
File path: modules/schema/src/test/java/org/apache/ignite/internal/schema/configuration/SchemaDescriptorConverterTest.java
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.internal.schema.Column;
+import org.apache.ignite.internal.schema.Columns;
+import org.apache.ignite.internal.schema.NativeTypeSpec;
+import org.apache.ignite.internal.schema.SchemaDescriptor;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.SchemaBuilders;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.builder.SchemaTableBuilder;
+import org.apache.ignite.schema.builder.TableColumnBuilder;
+import org.junit.jupiter.api.Test;
+
+import java.util.function.Function;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/** Tests for SchemaDescriptorConverter. */
+public class SchemaDescriptorConverterTest {
+    /** Totoal number of columns. */
+    private static final int columns = 14;
+
+    /**
+     * Convert table with complex primary key and check it.
+     */
+    @Test
+    public void testComplexPrimaryIndex() {
+        SchemaTableBuilder bldr = getBuilder(false, false);
+        SchemaTable tblSchm = bldr.withIndex(SchemaBuilders.pkIndex()
+            .addIndexColumn("INT8").done()
+            .addIndexColumn("ID").done()
+            .build()
+        ).build();
+
+        SchemaDescriptor tblDscr = SchemaDescriptorConverter.convert(tblSchm);
+
+        assertEquals(2, tblSchm.keyColumns().size());
+        assertEquals(columns - 2, tblSchm.valueColumns().size());
+    }
+
+    /**
+     * Convert table with nullable columns.
+     */
+    @Test
+    public void convertNullable() {
+        testConvert(true);
+    }
+
+    /**
+     * Convert table with non nullable columns.
+     */
+    @Test
+    public void convertTypes() {
+        testConvert(false);
+    }
+
+    /**
+     * Test set of columns.
+     *
+     * @param nullable Nullable flag.
+     */
+    private void testConvert(boolean nullable) {
+        SchemaTable tblSchm = getBuilder(nullable, true).build();
+
+        SchemaDescriptor tblDscr = SchemaDescriptorConverter.convert(tblSchm);
+
+        assertEquals(1, tblDscr.keyColumns().length());
+        testCol(tblDscr.keyColumns(), "ID", NativeTypeSpec.UUID, nullable);
+
+        assertEquals(columns - 1, tblDscr.valueColumns().length());
+        testCol(tblDscr.valueColumns(), "INT8", NativeTypeSpec.BYTE, nullable);
+        testCol(tblDscr.valueColumns(), "INT16", NativeTypeSpec.SHORT, nullable);
+        testCol(tblDscr.valueColumns(), "INT32", NativeTypeSpec.INTEGER, nullable);
+        testCol(tblDscr.valueColumns(), "INT64", NativeTypeSpec.LONG, nullable);
+        testCol(tblDscr.valueColumns(), "FLOAT", NativeTypeSpec.FLOAT, nullable);
+        testCol(tblDscr.valueColumns(), "DOUBLE", NativeTypeSpec.DOUBLE, nullable);
+        testCol(tblDscr.valueColumns(), "UUID", NativeTypeSpec.UUID, nullable);
+        testCol(tblDscr.valueColumns(), "STRING", NativeTypeSpec.STRING, nullable);
+        testCol(tblDscr.valueColumns(), "STRING_FS10", NativeTypeSpec.STRING, nullable);
+        testCol(tblDscr.valueColumns(), "BLOB", NativeTypeSpec.BYTES, nullable);
+        testCol(tblDscr.valueColumns(), "DECIMAL", NativeTypeSpec.DECIMAL, nullable);
+        testCol(tblDscr.valueColumns(), "BITMASK_FS10", NativeTypeSpec.BITMASK, nullable);
+    }
+
+    /**
+     * Get SchemaTableBuilder with default table.
+     *
+     * @param nullable If all columns should be nullable.
+     * @param withPk If builder should contains primary key index.
+     * @return SchemaTableBuilder.
+     */
+    private SchemaTableBuilder getBuilder(boolean nullable, boolean withPk) {
+        Function<TableColumnBuilder, org.apache.ignite.schema.Column> postProcess = builder -> {
+            if (nullable)
+                builder.asNullable();
+            else
+                builder.asNonNull();
+            return builder.build();
+        };
+
+        SchemaTableBuilder res = SchemaBuilders.tableBuilder("SCHEMA", "TABLE")
+            .columns(
+                postProcess.apply(SchemaBuilders.column("ID", ColumnType.UUID)),
+                postProcess.apply(SchemaBuilders.column("INT8", ColumnType.INT8)),
+                postProcess.apply(SchemaBuilders.column("INT16", ColumnType.INT16)),
+                postProcess.apply(SchemaBuilders.column("INT32", ColumnType.INT32)),
+                postProcess.apply(SchemaBuilders.column("INT64", ColumnType.INT64)),
+                postProcess.apply(SchemaBuilders.column("FLOAT", ColumnType.FLOAT)),
+                postProcess.apply(SchemaBuilders.column("DOUBLE", ColumnType.DOUBLE)),
+                postProcess.apply(SchemaBuilders.column("UUID", ColumnType.UUID)),
+                postProcess.apply(SchemaBuilders.column("STRING", ColumnType.string())),
+                postProcess.apply(SchemaBuilders.column("STRING_FS10", ColumnType.stringOf(10))),
+                postProcess.apply(SchemaBuilders.column("BLOB", ColumnType.blobOf())),
+                postProcess.apply(SchemaBuilders.column("BLOB_FS10", ColumnType.blobOf(10))),
+                postProcess.apply(SchemaBuilders.column("DECIMAL", ColumnType.number(1,1))),
+                postProcess.apply(SchemaBuilders.column("BITMASK_FS10", ColumnType.bitmaskOf(10)))
+                // TODO: uncomment after unsigned types available

Review comment:
       TODO with no  ticket




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

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



[GitHub] [ignite-3] AMashenkov commented on a change in pull request #132: GG-14290: SchemaDescriptor and ConfigurationSchema converters.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #132:
URL: https://github.com/apache/ignite-3/pull/132#discussion_r635031437



##########
File path: modules/schema/src/test/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverterTest.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.ConfigurationRegistry;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableValidator;
+import org.apache.ignite.configuration.schemas.table.TablesConfiguration;
+import org.apache.ignite.internal.schema.builder.HashIndexBuilderTest;
+import org.apache.ignite.internal.schema.builder.PartialIndexBuilderTest;
+import org.apache.ignite.internal.schema.builder.PrimaryKeyBuilderTest;
+import org.apache.ignite.internal.schema.builder.SchemaTableBuilderTest;
+import org.apache.ignite.internal.schema.builder.SortedIndexBuilderTest;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Collections;
+import java.util.concurrent.ExecutionException;
+
+import static org.apache.ignite.internal.schema.builder.SchemaTableBuilderTest.TBL;
+
+/** SchmConfigurationConverter tests. */
+public class SchemaConfigurationConverterTest {
+    /** Configuration registry with one table for each test. */
+    private ConfigurationRegistry confRegistry;
+
+    /**
+     * Prepare configuration registry for test.
+     *
+     * @throws ExecutionException
+     * @throws InterruptedException
+     */
+    @BeforeEach
+    public void createRegistry() throws ExecutionException, InterruptedException {
+        confRegistry = new ConfigurationRegistry(
+            Collections.singleton(TablesConfiguration.KEY),
+            Collections.singletonMap(TableValidator.class, Collections.singleton(SchemaTableValidatorImpl.INSTANCE)),
+            Collections.singleton(new TestConfigurationStorage()));
+
+        confRegistry.getConfiguration(TablesConfiguration.KEY).change(
+            ch -> {
+                SchemaConfigurationConverter.createTable(TBL, ch);
+                ch.changeTables(tblsCh -> tblsCh.create(TBL.canonicalName(),
+                    tblCh -> tblCh.changeReplicas(1)));
+            }).get();
+    }
+
+    /**
+     * Add/remove HashIndex into configuration and read it back.
+     */
+    @Test
+    public void testConvertHashIndex() throws ExecutionException, InterruptedException {
+        HashIndex idx = HashIndexBuilderTest.IDX;
+        getTbl().change(ch -> SchemaConfigurationConverter.addIndex(idx, ch)).get();
+    }
+
+    /**
+     * Add/remove SortedIndex into configuration and read it back.
+     */
+    @Test
+    public void testConvertSortedIndex() throws ExecutionException, InterruptedException {
+        SortedIndex idx = SortedIndexBuilderTest.IDX;
+
+        getTbl().change(ch -> SchemaConfigurationConverter.addIndex(idx, ch)).get();
+    }
+
+    /**
+     * Add/remove PrimaryIndex into configuration and read it back.
+     */
+    @Test
+    public void testPKIndex() throws ExecutionException, InterruptedException {
+        PrimaryIndex idx = PrimaryKeyBuilderTest.IDX;
+
+        getTbl().change(ch -> SchemaConfigurationConverter.addIndex(idx, ch)).get();
+    }
+
+    /**
+     * Add/remove PartialIndex into configuration and read it back.
+     */
+    @Test
+    public void testPartialIndex() throws ExecutionException, InterruptedException {
+        PartialIndex idx = PartialIndexBuilderTest.IDX;
+
+        getTbl().change(ch -> SchemaConfigurationConverter.addIndex(idx, ch)).get();
+    }
+
+    /**
+     * Add/remove table and read it back.
+     */
+    @Test
+    public void testConvertTable() {
+        // TODO

Review comment:
       TODO with no ticket




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

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



[GitHub] [ignite-3] AMashenkov commented on a change in pull request #132: GG-14290: SchemaDescriptor and ConfigurationSchema converters.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #132:
URL: https://github.com/apache/ignite-3/pull/132#discussion_r635022015



##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnChange;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexChange;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.schemas.table.TablesChange;
+import org.apache.ignite.configuration.tree.NamedListView;
+import org.apache.ignite.internal.schema.ColumnImpl;
+import org.apache.ignite.internal.schema.HashIndexImpl;
+import org.apache.ignite.internal.schema.PartialIndexImpl;
+import org.apache.ignite.internal.schema.PrimaryIndexImpl;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.SortedIndexColumnImpl;
+import org.apache.ignite.internal.schema.SortedIndexImpl;
+import org.apache.ignite.schema.Column;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.IndexColumn;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
+import org.apache.ignite.schema.SortedIndexColumn;
+import org.apache.ignite.schema.TableIndex;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/** Configuration to schema and vice versa converter. */
+public class SchemaConfigurationConverter {
+    /** Hash index type. */
+    private static final String HASH_TYPE = "HASH";
+
+    /** Sorted index type. */
+    private static final String SORTED_TYPE = "SORTED";
+
+    /** Partial index type. */
+    private static final String PARTIAL_TYPE = "PARTIAL";
+
+    /** Primary key index type. */
+    private static final String PK_TYPE = "PK";
+
+    /** Types map. */
+    private static Map<String, ColumnType> types = new HashMap<>();
+
+    static {
+        putType(ColumnType.INT8);
+        putType(ColumnType.INT16);
+        putType(ColumnType.INT32);
+        putType(ColumnType.INT64);
+        putType(ColumnType.UINT8);
+        putType(ColumnType.UINT16);
+        putType(ColumnType.UINT32);
+        putType(ColumnType.UINT64);
+        putType(ColumnType.FLOAT);
+        putType(ColumnType.DOUBLE);
+        putType(ColumnType.UUID);
+        putType(ColumnType.string());
+        putType(ColumnType.blobOf());
+        // TODO; handle length for some types
+        //putType(ColumnType.bitmaskOf());
+    }
+
+    private static void putType(ColumnType type) {
+        types.put(type.typeSpec().name(), type);
+    }
+
+    /**
+     * Convert SortedIndexColumn to IndexColumnChange.
+     * @param col IndexColumnChange.
+     * @param colInit IndexColumnChange to fullfill.
+     */
+    public static void convert(SortedIndexColumn col, IndexColumnChange colInit) {
+        colInit.changeName(col.name());
+        colInit.changeAsc(col.asc());
+    }
+
+    /**
+     * Convert IndexColumnView to SortedIndexColumn.
+     *
+     * @param colCfg IndexColumnView.
+     * @return SortedIndexColumn.
+     */
+    public static SortedIndexColumn convert(IndexColumnView colCfg) {
+        return new SortedIndexColumnImpl(colCfg.name(), colCfg.asc());
+    }
+
+    /**
+     * Convert TableIndex to TableIndexChange.
+     *
+     * @param idx TableIndex.
+     * @param idxChg TableIndexChange to fullfill.
+     */
+    public static void convert(TableIndex idx, TableIndexChange idxChg) {
+        idxChg.changeName(idx.name());
+        idxChg.changeType(idx.type());
+
+        if (HASH_TYPE.equals(idx.type())) {
+            HashIndex hashIdx = (HashIndex)idx;
+
+            String[] colNames = hashIdx.columns().stream().map(IndexColumn::name).toArray(String[]::new);
+            idxChg.changeColNames(colNames);
+        }
+        else if (PARTIAL_TYPE.equals(idx.type())) {
+            PartialIndex partIdx = (PartialIndex)idx;
+
+            idxChg.changeUniq(partIdx.unique());
+            idxChg.changeExpr(partIdx.expr());
+
+            idxChg.changeColumns(colsChg -> {
+                for (SortedIndexColumn col : partIdx.columns())
+                    colsChg.create(col.name(), colInit -> convert(col, colInit));
+            });
+
+        }
+        else if (SORTED_TYPE.equals(idx.type())) {
+            SortedIndex sortIdx = (SortedIndex)idx;
+            idxChg.changeUniq(sortIdx.unique());
+
+            idxChg.changeColumns(colsInit -> {
+                for (SortedIndexColumn col : sortIdx.columns())
+                    colsInit.create(col.name(), colInit -> convert(col, colInit));
+            });
+        }
+        else if (PK_TYPE.equals(idx.type())) {
+            PrimaryIndex primIdx = (PrimaryIndex)idx;
+
+            idxChg.changeColumns(colsInit -> {
+                for (SortedIndexColumn col : primIdx.columns())
+                    colsInit.create(col.name(), colInit -> convert(col, colInit));
+            });
+
+            idxChg.changeAffinityColumns(primIdx.affinityColumns().toArray(new String[primIdx.affinityColumns().size()]));
+        }
+        else throw new IllegalArgumentException("Unknown index type " + idx.type());
+    }
+
+    /**
+     * Convert TableIndexView into TableIndex.
+     *
+     * @param idxView TableIndexView.
+     * @return TableIndex.
+     */
+    public static TableIndex convert(TableIndexView idxView) {
+        String name = idxView.name();
+        String type = idxView.type();
+
+        if (type.equals("HASH")) {
+            String[] cols = idxView.colNames();
+
+            return new HashIndexImpl(name, cols);
+        }
+        else if (type.equals("SORTED") || type.equals("PARTIAL") || type.equals("PK")) {
+            if (type.equals("PARTIAL")) {
+                boolean uniq = idxView.uniq();
+                String expr = idxView.expr();
+                NamedListView<? extends IndexColumnView> colsView = idxView.columns();
+                List<SortedIndexColumn> cols = new ArrayList<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(colsView.get(key));
+                    cols.add(col);
+                }
+
+                return new PartialIndexImpl(name, cols, uniq, expr);
+            }
+            if (type.equals("SORTED")) {
+                boolean uniq = idxView.uniq();
+                List<SortedIndexColumn> cols = new ArrayList<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    cols.add(col);
+                }
+
+                return new SortedIndexImpl(name, cols, uniq);
+            }
+            if (type.equals("PK")) {
+                List<SortedIndexColumn> cols = new ArrayList<>();
+                for (String key : idxView.columns().namedListKeys()) {
+                    SortedIndexColumn col = convert(idxView.columns().get(key));
+                    cols.add(col);
+                }
+                String[] affCols = idxView.colNames();
+
+                return new PrimaryIndexImpl(cols, Arrays.stream(affCols).collect(Collectors.toList()));
+            }
+
+            return null;
+        }
+        else
+            throw new IllegalArgumentException("Unknown type " + type);
+    }
+
+    /**
+     * Convert ColumnType to ColumnTypeChange.
+     *
+     * @param colType ColumnType.
+     * @param colTypeChg ColumnTypeChange to fullfill.
+     */
+    public static void convert(ColumnType colType, ColumnTypeChange colTypeChg) {
+        colTypeChg.changeType(colType.typeSpec().name());
+        // TODO varlen types
+    }
+
+    /**
+     * Convert ColumnTypeView to ColumnType.
+     *
+     * @param colTypeView ColumnTypeView.
+     * @return ColumnType.
+     */
+    public static ColumnType convert(ColumnTypeView colTypeView) {
+        String typeName = colTypeView.type();
+        ColumnType res = types.get(typeName);
+        if (res != null)
+            return res;
+        else
+            // TODO varlen types
+            throw new IllegalArgumentException();
+    }
+
+    /**
+     * Convert column to column change.
+     *
+     * @param col Column to convert.
+     * @param colChg Column
+     */
+    public static void convert(Column col, ColumnChange colChg) {
+        colChg.changeName(col.name());
+        colChg.changeType(colTypeInit -> convert(col.type(), colTypeInit));
+        if (col.defaultValue() != null)
+            colChg.changeDefaultValue(col.defaultValue().toString()); // TODO: specify "default type" type
+        colChg.changeNullable(col.nullable());
+    }
+
+    /**
+     * Convert column view to Column.
+     *
+     * @param colView Column view.
+     * @return Column.
+     */
+    public static Column convert(ColumnView colView) {
+        String name = colView.name();
+        ColumnType type = convert(colView.type());
+        boolean nullable = colView.nullable();
+        String defValue = colView.defaultValue();
+
+        return new ColumnImpl(name, type, nullable, defValue);

Review comment:
       ```suggestion       
           return new ColumnImpl(
               colView.name(),
               convert(colView.type()),
               colView.nullable(),
               colView.defaultValue());
   ```




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

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



[GitHub] [ignite-3] AMashenkov commented on a change in pull request #132: GG-14290: SchemaDescriptor and ConfigurationSchema converters.

Posted by GitBox <gi...@apache.org>.
AMashenkov commented on a change in pull request #132:
URL: https://github.com/apache/ignite-3/pull/132#discussion_r635018936



##########
File path: modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
##########
@@ -0,0 +1,397 @@
+/*
+ * 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.schema.configuration;
+
+import org.apache.ignite.configuration.schemas.table.ColumnChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeChange;
+import org.apache.ignite.configuration.schemas.table.ColumnTypeView;
+import org.apache.ignite.configuration.schemas.table.ColumnView;
+import org.apache.ignite.configuration.schemas.table.IndexColumnChange;
+import org.apache.ignite.configuration.schemas.table.IndexColumnView;
+import org.apache.ignite.configuration.schemas.table.TableChange;
+import org.apache.ignite.configuration.schemas.table.TableConfiguration;
+import org.apache.ignite.configuration.schemas.table.TableIndexChange;
+import org.apache.ignite.configuration.schemas.table.TableIndexView;
+import org.apache.ignite.configuration.schemas.table.TableView;
+import org.apache.ignite.configuration.schemas.table.TablesChange;
+import org.apache.ignite.configuration.tree.NamedListView;
+import org.apache.ignite.internal.schema.ColumnImpl;
+import org.apache.ignite.internal.schema.HashIndexImpl;
+import org.apache.ignite.internal.schema.PartialIndexImpl;
+import org.apache.ignite.internal.schema.PrimaryIndexImpl;
+import org.apache.ignite.internal.schema.SchemaTableImpl;
+import org.apache.ignite.internal.schema.SortedIndexColumnImpl;
+import org.apache.ignite.internal.schema.SortedIndexImpl;
+import org.apache.ignite.schema.Column;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.IndexColumn;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
+import org.apache.ignite.schema.SortedIndexColumn;
+import org.apache.ignite.schema.TableIndex;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+/** Configuration to schema and vice versa converter. */
+public class SchemaConfigurationConverter {
+    /** Hash index type. */
+    private static final String HASH_TYPE = "HASH";
+
+    /** Sorted index type. */
+    private static final String SORTED_TYPE = "SORTED";
+
+    /** Partial index type. */
+    private static final String PARTIAL_TYPE = "PARTIAL";
+
+    /** Primary key index type. */
+    private static final String PK_TYPE = "PK";
+
+    /** Types map. */
+    private static Map<String, ColumnType> types = new HashMap<>();
+
+    static {
+        putType(ColumnType.INT8);
+        putType(ColumnType.INT16);
+        putType(ColumnType.INT32);
+        putType(ColumnType.INT64);
+        putType(ColumnType.UINT8);
+        putType(ColumnType.UINT16);
+        putType(ColumnType.UINT32);
+        putType(ColumnType.UINT64);
+        putType(ColumnType.FLOAT);
+        putType(ColumnType.DOUBLE);
+        putType(ColumnType.UUID);
+        putType(ColumnType.string());
+        putType(ColumnType.blobOf());
+        // TODO; handle length for some types
+        //putType(ColumnType.bitmaskOf());
+    }
+
+    private static void putType(ColumnType type) {
+        types.put(type.typeSpec().name(), type);
+    }
+
+    /**
+     * Convert SortedIndexColumn to IndexColumnChange.
+     * @param col IndexColumnChange.
+     * @param colInit IndexColumnChange to fullfill.
+     */
+    public static void convert(SortedIndexColumn col, IndexColumnChange colInit) {
+        colInit.changeName(col.name());
+        colInit.changeAsc(col.asc());
+    }
+
+    /**
+     * Convert IndexColumnView to SortedIndexColumn.
+     *
+     * @param colCfg IndexColumnView.
+     * @return SortedIndexColumn.
+     */
+    public static SortedIndexColumn convert(IndexColumnView colCfg) {
+        return new SortedIndexColumnImpl(colCfg.name(), colCfg.asc());
+    }
+
+    /**
+     * Convert TableIndex to TableIndexChange.
+     *
+     * @param idx TableIndex.
+     * @param idxChg TableIndexChange to fullfill.
+     */
+    public static void convert(TableIndex idx, TableIndexChange idxChg) {
+        idxChg.changeName(idx.name());
+        idxChg.changeType(idx.type());
+
+        if (HASH_TYPE.equals(idx.type())) {
+            HashIndex hashIdx = (HashIndex)idx;
+
+            String[] colNames = hashIdx.columns().stream().map(IndexColumn::name).toArray(String[]::new);
+            idxChg.changeColNames(colNames);
+        }
+        else if (PARTIAL_TYPE.equals(idx.type())) {
+            PartialIndex partIdx = (PartialIndex)idx;
+
+            idxChg.changeUniq(partIdx.unique());
+            idxChg.changeExpr(partIdx.expr());
+
+            idxChg.changeColumns(colsChg -> {
+                for (SortedIndexColumn col : partIdx.columns())
+                    colsChg.create(col.name(), colInit -> convert(col, colInit));
+            });
+
+        }
+        else if (SORTED_TYPE.equals(idx.type())) {
+            SortedIndex sortIdx = (SortedIndex)idx;
+            idxChg.changeUniq(sortIdx.unique());
+
+            idxChg.changeColumns(colsInit -> {
+                for (SortedIndexColumn col : sortIdx.columns())
+                    colsInit.create(col.name(), colInit -> convert(col, colInit));
+            });
+        }
+        else if (PK_TYPE.equals(idx.type())) {
+            PrimaryIndex primIdx = (PrimaryIndex)idx;
+
+            idxChg.changeColumns(colsInit -> {
+                for (SortedIndexColumn col : primIdx.columns())
+                    colsInit.create(col.name(), colInit -> convert(col, colInit));
+            });
+
+            idxChg.changeAffinityColumns(primIdx.affinityColumns().toArray(new String[primIdx.affinityColumns().size()]));
+        }
+        else throw new IllegalArgumentException("Unknown index type " + idx.type());
+    }
+
+    /**
+     * Convert TableIndexView into TableIndex.
+     *
+     * @param idxView TableIndexView.
+     * @return TableIndex.
+     */
+    public static TableIndex convert(TableIndexView idxView) {
+        String name = idxView.name();
+        String type = idxView.type();
+
+        if (type.equals("HASH")) {
+            String[] cols = idxView.colNames();
+
+            return new HashIndexImpl(name, cols);
+        }
+        else if (type.equals("SORTED") || type.equals("PARTIAL") || type.equals("PK")) {
+            if (type.equals("PARTIAL")) {

Review comment:
       Let's rewrite to 'else-if'




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

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