You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by am...@apache.org on 2021/05/24 09:46:34 UTC

[ignite-3] branch main updated: IGNITE-14290: Schema configuration API implementation. (#132)

This is an automated email from the ASF dual-hosted git repository.

amashenkov pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git


The following commit(s) were added to refs/heads/main by this push:
     new 26839e9  IGNITE-14290: Schema configuration API implementation. (#132)
26839e9 is described below

commit 26839e9b3936226b3a6421b6d2510cb74bfb3f1e
Author: Berkof <sa...@mail.ru>
AuthorDate: Mon May 24 16:46:25 2021 +0700

    IGNITE-14290: Schema configuration API implementation. (#132)
---
 .../schemas/table/ColumnConfigurationSchema.java   |   4 +-
 .../table/ColumnTypeConfigurationSchema.java       |   4 +-
 .../table/IndexColumnConfigurationSchema.java      |   4 +-
 .../table/TableIndexConfigurationSchema.java       |   4 +-
 ...onfigurationSchema.java => TableValidator.java} |  31 +-
 .../schemas/table/TablesConfigurationSchema.java   |   6 +-
 .../apache/ignite/internal/schema/NativeTypes.java |  16 +-
 .../org/apache/ignite/internal/schema/Row.java     |   9 +-
 .../ignite/internal/schema/SchemaManager.java      |  86 +---
 .../schema/builder/SchemaTableBuilderImpl.java     |  28 +-
 .../SchemaConfigurationConverter.java              | 499 +++++++++++++++++++++
 .../configuration/SchemaDescriptorConverter.java   | 140 ++++++
 .../configuration/SchemaTableValidatorImpl.java    |  66 +++
 .../schema/builder/HashIndexBuilderTest.java       |  57 +++
 .../schema/builder/PartialIndexBuilderTest.java}   |  44 +-
 .../schema/builder/PrimaryKeyBuilderTest.java      |  47 ++
 .../schema/builder/SchemaTableBuilderTest.java     |  49 ++
 .../schema/builder/SortedIndexBuilderTest.java     |  49 ++
 .../schema/builder/TableColumnBuilderTest.java     |  47 ++
 .../SchemaConfigurationConverterTest.java          | 224 +++++++++
 .../SchemaDescriptorConverterTest.java             | 190 ++++++++
 .../configuration/TestConfigurationStorage.java    |  69 +++
 22 files changed, 1534 insertions(+), 139 deletions(-)

diff --git a/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/ColumnConfigurationSchema.java b/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/ColumnConfigurationSchema.java
index 69fc03a..bf4f9ea 100644
--- a/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/ColumnConfigurationSchema.java
+++ b/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/ColumnConfigurationSchema.java
@@ -22,7 +22,9 @@ import org.apache.ignite.configuration.annotation.ConfigValue;
 import org.apache.ignite.configuration.annotation.Value;
 import org.apache.ignite.configuration.validation.Immutable;
 
-/** Configuration for single column in SQL table. */
+/**
+ * Configuration for single column in SQL table.
+ */
 @Config
 public class ColumnConfigurationSchema {
     /** Column name. */
diff --git a/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/ColumnTypeConfigurationSchema.java b/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/ColumnTypeConfigurationSchema.java
index ccb02ce..2a17e1b 100644
--- a/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/ColumnTypeConfigurationSchema.java
+++ b/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/ColumnTypeConfigurationSchema.java
@@ -20,7 +20,9 @@ package org.apache.ignite.configuration.schemas.table;
 import org.apache.ignite.configuration.annotation.Config;
 import org.apache.ignite.configuration.annotation.Value;
 
-/** Configuration for SQL table column type. */
+/**
+ * Configuration for SQL table column type.
+ */
 @Config
 public class ColumnTypeConfigurationSchema {
     /** Type name. */
diff --git a/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/IndexColumnConfigurationSchema.java b/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/IndexColumnConfigurationSchema.java
index 3cf584c..8651ed7 100644
--- a/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/IndexColumnConfigurationSchema.java
+++ b/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/IndexColumnConfigurationSchema.java
@@ -21,7 +21,9 @@ import org.apache.ignite.configuration.annotation.Config;
 import org.apache.ignite.configuration.annotation.Value;
 import org.apache.ignite.configuration.validation.Immutable;
 
-/** Configuration for single column in index. */
+/**
+ * Configuration for single column in index.
+ */
 @Config
 public class IndexColumnConfigurationSchema {
     /** Column name. */
diff --git a/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/TableIndexConfigurationSchema.java b/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/TableIndexConfigurationSchema.java
index 7d90331..4105cd2 100644
--- a/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/TableIndexConfigurationSchema.java
+++ b/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/TableIndexConfigurationSchema.java
@@ -22,7 +22,9 @@ import org.apache.ignite.configuration.annotation.NamedConfigValue;
 import org.apache.ignite.configuration.annotation.Value;
 import org.apache.ignite.configuration.validation.Immutable;
 
-/** SQL index configuration. */
+/**
+ * SQL index configuration.
+ */
 @Config
 public class TableIndexConfigurationSchema {
     /** Index name. */
diff --git a/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/ColumnTypeConfigurationSchema.java b/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/TableValidator.java
similarity index 63%
copy from modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/ColumnTypeConfigurationSchema.java
copy to modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/TableValidator.java
index ccb02ce..ed0d65d 100644
--- a/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/ColumnTypeConfigurationSchema.java
+++ b/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/TableValidator.java
@@ -17,25 +17,20 @@
 
 package org.apache.ignite.configuration.schemas.table;
 
-import org.apache.ignite.configuration.annotation.Config;
-import org.apache.ignite.configuration.annotation.Value;
+import java.lang.annotation.Retention;
+import java.lang.annotation.Target;
 
-/** Configuration for SQL table column type. */
-@Config
-public class ColumnTypeConfigurationSchema {
-    /** Type name. */
-    @Value
-    String type;
+import static java.lang.annotation.ElementType.FIELD;
+import static java.lang.annotation.ElementType.PARAMETER;
+import static java.lang.annotation.RetentionPolicy.RUNTIME;
 
-    /** Length. */
-    @Value(hasDefault = true)
-    int length = 0;
-
-    /** Precision. */
-    @Value(hasDefault = true)
-    int precision = 0;
+/**
+ * Annotation to validate whole table configuration.
+ *
+ * Activate SchemaTableValidatorImpl in configuration engine for {@link TablesConfigurationSchema#tables}.
+ */
+@Target({ FIELD, PARAMETER })
+@Retention(RUNTIME)
+public @interface TableValidator {
 
-    /** Scale. */
-    @Value(hasDefault = true)
-    int scale = 0;
 }
diff --git a/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/TablesConfigurationSchema.java b/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/TablesConfigurationSchema.java
index 3ed2539..c5fc29b 100644
--- a/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/TablesConfigurationSchema.java
+++ b/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/TablesConfigurationSchema.java
@@ -21,12 +21,14 @@ import org.apache.ignite.configuration.annotation.ConfigurationRoot;
 import org.apache.ignite.configuration.annotation.NamedConfigValue;
 import org.apache.ignite.configuration.storage.ConfigurationType;
 
-/** Tables configuration schema. */
+/**
+ * Tables configuration schema.
+ */
 @SuppressWarnings("PMD.UnusedPrivateField")
 @ConfigurationRoot(rootName = "table", type = ConfigurationType.DISTRIBUTED)
 public class TablesConfigurationSchema {
-
     /** List of configured tables. */
     @NamedConfigValue
+    @TableValidator
     TableConfigurationSchema tables;
 }
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/NativeTypes.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/NativeTypes.java
index 53ebe75..bdd7809 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/NativeTypes.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/NativeTypes.java
@@ -86,6 +86,16 @@ public class NativeTypes {
     }
 
     /**
+     * Creates a DECIMAL type with maximal precision and scale.
+     *
+     * @param precision Precision.
+     * @param scale Scale.
+     */
+    public static NativeType decimalOf(int precision, int scale) {
+        return new NumericNativeType(precision, scale);
+    }
+
+    /**
      * Return the native type for specified object.
      *
      * @param val Object to map to native type.
@@ -120,13 +130,13 @@ public class NativeTypes {
                 return UUID;
 
             case STRING:
-                return NativeTypes.stringOf(((CharSequence)val).length());
+                return stringOf(((CharSequence)val).length());
 
             case BYTES:
-                return NativeTypes.blobOf(((byte[])val).length);
+                return blobOf(((byte[])val).length);
 
             case BITMASK:
-                return NativeTypes.bitmaskOf(((BitSet)val).length());
+                return bitmaskOf(((BitSet)val).length());
 
             default:
                 assert false : "Unexpected type: " + spec;
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/Row.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/Row.java
index c3bb8c5..a2fa63e 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/Row.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/Row.java
@@ -229,7 +229,14 @@ public class Row implements BinaryRow {
         return off < 0 ? null : readDouble(offset(off));
     }
 
-    public BigDecimal decimalValue(int idx) {
+    /**
+     * Reads value from specified column.
+     *
+     * @param col Column index.
+     * @return Column value.
+     * @throws InvalidTypeException If actual column type does not match the requested column type.
+     */
+    public BigDecimal decimalValue(int col) throws InvalidTypeException {
         // TODO: IGNITE-13668 decimal support
         return null;
     }
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/SchemaManager.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/SchemaManager.java
index 24994c3..f388859 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/SchemaManager.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/SchemaManager.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.schema;
 
-import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Set;
@@ -25,17 +24,13 @@ import java.util.UUID;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ExecutionException;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
 import org.apache.ignite.configuration.internal.ConfigurationManager;
-import org.apache.ignite.configuration.schemas.table.ColumnTypeView;
-import org.apache.ignite.configuration.schemas.table.ColumnView;
 import org.apache.ignite.configuration.schemas.table.TableConfiguration;
-import org.apache.ignite.configuration.schemas.table.TableIndexConfiguration;
 import org.apache.ignite.configuration.schemas.table.TablesConfiguration;
-import org.apache.ignite.configuration.tree.NamedListView;
 import org.apache.ignite.internal.manager.Producer;
 import org.apache.ignite.internal.metastorage.MetaStorageManager;
+import org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter;
+import org.apache.ignite.internal.schema.configuration.SchemaDescriptorConverter;
 import org.apache.ignite.internal.schema.event.SchemaEvent;
 import org.apache.ignite.internal.schema.event.SchemaEventParameters;
 import org.apache.ignite.internal.schema.registry.SchemaRegistryException;
@@ -51,7 +46,7 @@ import org.apache.ignite.metastorage.client.EntryEvent;
 import org.apache.ignite.metastorage.client.Operations;
 import org.apache.ignite.metastorage.client.WatchEvent;
 import org.apache.ignite.metastorage.client.WatchListener;
-import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaTable;
 import org.jetbrains.annotations.NotNull;
 
 /**
@@ -180,7 +175,8 @@ public class SchemaManager extends Producer<SchemaEvent, SchemaEventParameters>
                 final ByteArray lastVerKey = new ByteArray(INTERNAL_PREFIX + tblId);
                 final ByteArray schemaKey = new ByteArray(INTERNAL_PREFIX + tblId + INTERNAL_VER_SUFFIX + schemaVer);
 
-                final SchemaDescriptor desc = createSchemaDescriptor(tblId, schemaVer, tblConfig);
+                SchemaTable schemaTable = SchemaConfigurationConverter.convert(tblConfig);
+                final SchemaDescriptor desc = SchemaDescriptorConverter.convert(tblId, schemaVer, schemaTable);
 
                 return metaStorageMgr.invoke(Conditions.notExists(schemaKey),
                     Operations.put(schemaKey, ByteUtils.toBytes(desc)),
@@ -210,78 +206,6 @@ public class SchemaManager extends Producer<SchemaEvent, SchemaEventParameters>
     }
 
     /**
-     * Creates schema descriptor from configuration.
-     *
-     * @param tblId Table ID.
-     * @param ver Schema version.
-     * @param tblConfig Table config.
-     * @return Schema descriptor.
-     */
-    private SchemaDescriptor createSchemaDescriptor(UUID tblId, int ver, TableConfiguration tblConfig) {
-        final TableIndexConfiguration pkCfg = tblConfig.indices().get(PrimaryIndex.PRIMARY_KEY_INDEX_NAME);
-
-        assert pkCfg != null;
-
-        final Set<String> keyColNames = Stream.of(pkCfg.colNames().value()).collect(Collectors.toSet());
-        final NamedListView<ColumnView> cols = tblConfig.columns().value();
-
-        final ArrayList<Column> keyCols = new ArrayList<>(keyColNames.size());
-        final ArrayList<Column> valCols = new ArrayList<>(cols.size() - keyColNames.size());
-
-        cols.namedListKeys().stream()
-            .map(cols::get)
-            //TODO: IGNITE-14290 replace with helper class call.
-            .map(col -> new Column(col.name(), createType(col.type()), col.nullable()))
-            .forEach(c -> (keyColNames.contains(c.name()) ? keyCols : valCols).add(c));
-
-        return new SchemaDescriptor(
-            tblId,
-            ver,
-            keyCols.toArray(Column[]::new),
-            pkCfg.affinityColumns().value(),
-            valCols.toArray(Column[]::new)
-        );
-    }
-
-    /**
-     * Create type from config.
-     *
-     * TODO: IGNITE-14290 replace with helper class call.
-     *
-     * @param type Type view.
-     * @return Native type.
-     */
-    private NativeType createType(ColumnTypeView type) {
-        switch (type.type().toLowerCase()) {
-            case "int8":
-                return NativeTypes.BYTE;
-            case "int16":
-                return NativeTypes.SHORT;
-            case "int32":
-                return NativeTypes.INTEGER;
-            case "int64":
-                return NativeTypes.LONG;
-            case "float":
-                return NativeTypes.FLOAT;
-            case "double":
-                return NativeTypes.DOUBLE;
-            case "uuid":
-                return NativeTypes.UUID;
-            case "bitmask":
-                assert type.length() > 0;
-
-                return NativeTypes.bitmaskOf(type.length());
-            case "string":
-                return type.length() == 0 ? NativeTypes.STRING : NativeTypes.stringOf(type.length());
-            case "bytes":
-                return type.length() == 0 ? NativeTypes.BYTES : NativeTypes.blobOf(type.length());
-
-            default:
-                throw new IllegalStateException("Unsupported column type: " + type.type());
-        }
-    }
-
-    /**
      * Compares schemas.
      *
      * @param expected Expected schema.
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/builder/SchemaTableBuilderImpl.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/builder/SchemaTableBuilderImpl.java
index 1c5f063..1da11d9 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/builder/SchemaTableBuilderImpl.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/builder/SchemaTableBuilderImpl.java
@@ -17,10 +17,14 @@
 
 package org.apache.ignite.internal.schema.builder;
 
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
 import org.apache.ignite.internal.schema.SchemaTableImpl;
 import org.apache.ignite.schema.Column;
 import org.apache.ignite.schema.ColumnarIndex;
@@ -28,6 +32,7 @@ import org.apache.ignite.schema.IndexColumn;
 import org.apache.ignite.schema.PrimaryIndex;
 import org.apache.ignite.schema.SchemaBuilders;
 import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.SortedIndex;
 import org.apache.ignite.schema.TableIndex;
 import org.apache.ignite.schema.builder.SchemaTableBuilder;
 
@@ -87,7 +92,7 @@ public class SchemaTableBuilderImpl implements SchemaTableBuilder {
 
     /** {@inheritDoc} */
     @Override public SchemaTableBuilder withPrimaryKey(String colName) {
-        withIndex(SchemaBuilders.pkIndex().addIndexColumn(colName).done().build());
+        withIndex(SchemaBuilders.pkIndex().addIndexColumn(colName).done().withAffinityColumns(colName).build());
 
         return this;
     }
@@ -101,9 +106,10 @@ public class SchemaTableBuilderImpl implements SchemaTableBuilder {
     /** {@inheritDoc} */
     @Override public SchemaTable build() {
         assert schemaName != null : "Table name was not specified.";
-        assert columns.size() >= 2 : "Key or/and value columns was not defined.";
 
-        validateIndices();
+        validateIndices(indices.values(), columns.values());
+
+        assert columns.size() > ((SortedIndex)indices.get(PRIMARY_KEY_INDEX_NAME)).columns().size() : "Key or/and value columns was not defined.";
 
         return new SchemaTableImpl(
             schemaName,
@@ -116,18 +122,22 @@ public class SchemaTableBuilderImpl implements SchemaTableBuilder {
     /**
      * 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.";
+
+        TableIndex pkIdx = indices.stream().filter(idx -> PRIMARY_KEY_INDEX_NAME.equals(idx.name())).findAny().orElse(null);
 
-        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.";
+        assert pkIdx != null : "Primary key index is not configured.";
+        assert !((PrimaryIndex)pkIdx).affinityColumns().isEmpty() : "Primary key must have one affinity column at least.";
 
         // Note: E.g. functional index is not columnar index as it index an expression result only.
-        assert indices.values().stream().allMatch(ColumnarIndex.class::isInstance) : "Columnar indices are supported only.";
+        assert indices.stream().allMatch(ColumnarIndex.class::isInstance) : "Columnar indices are supported only.";
     }
 }
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
new file mode 100644
index 0000000..f480d0d
--- /dev/null
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverter.java
@@ -0,0 +1,499 @@
+/*
+ * 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 final 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().toUpperCase()) {
+            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.toUpperCase()) {
+            case HASH_TYPE:
+                String[] hashCols = idxView.colNames();
+
+                return new HashIndexImpl(name, hashCols);
+
+            case SORTED_TYPE:
+                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_TYPE:
+                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_TYPE:
+                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().toUpperCase();
+        
+        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().toUpperCase();
+        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) {
+        return new ColumnImpl(
+            colView.name(),
+            convert(colView.type()),
+            colView.nullable(),
+            colView.defaultValue());
+    }
+
+    /**
+     * 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();
+        int sepPos = canonicalName.indexOf('.');
+        String schemaName = canonicalName.substring(0, sepPos);
+        String tableName = canonicalName.substring(sepPos + 1);
+
+        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);
+        }
+
+        LinkedHashMap<String, Column> colsMap = new LinkedHashMap<>(colsView.size());
+        
+        columns.forEach((i,v) -> colsMap.put(v.name(), v));
+
+        return new SchemaTableImpl(schemaName, tableName, colsMap, indices);
+    }
+
+    /**
+     * Create table.
+     *
+     * @param tbl Table to create.
+     * @param tblsChange Tables change to fulfill.
+     * @return TablesChange to get result from.
+     */
+    public static TablesChange createTable(SchemaTable tbl, TablesChange tblsChange) {
+        return tblsChange.changeTables(tblsChg -> tblsChg.create(tbl.canonicalName(), tblChg -> convert(tbl, tblChg)));
+    }
+
+    /**
+     * Drop table.
+     *
+     * @param tbl table to drop.
+     * @param tblsChange TablesChange change to fulfill.
+     * @return TablesChange to get result from.
+     */
+    public static TablesChange dropTable(SchemaTable tbl, TablesChange tblsChange) {
+        return tblsChange.changeTables(schmTblChange -> schmTblChange.delete(tbl.canonicalName()));
+    }
+
+    /**
+     * Add index.
+     *
+     * @param idx Index to add.
+     * @param tblChange TableChange to fullfill.
+     * @return TableChange to get result from.
+     */
+    public static TableChange addIndex(TableIndex idx, TableChange tblChange) {
+        return tblChange.changeIndices(idxsChg -> idxsChg.create(idx.name(), idxChg -> convert(idx, idxChg)));
+    }
+
+    /**
+     * Drop index.
+     *
+     * @param indexName Index name to drop.
+     * @param tblChange Table change to fulfill.
+     * @return TableChange to get result from.
+     */
+    public static TableChange dropIndex(String indexName, TableChange tblChange) {
+        return tblChange.changeIndices(idxChg -> idxChg.delete(indexName));
+    }
+
+    /**
+     * Add table column.
+     *
+     * @param column Column to add.
+     * @param tblChange TableChange to fulfill.
+     * @return TableChange to get result from.
+     */
+    public static TableChange addColumn(Column column, TableChange tblChange) {
+        return tblChange.changeColumns(colsChg -> colsChg.create(column.name(), colChg -> convert(column, colChg)));
+    }
+
+    /**
+     * Drop table column.
+     *
+     * @param columnName column name to drop.
+     * @param tblChange TableChange to fulfill.
+     * @return TableChange to get result from.
+     */
+    public static TableChange dropColumn(String columnName, TableChange tblChange) {
+        return tblChange.changeColumns(colChg -> colChg.delete(columnName));
+    }
+}
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaDescriptorConverter.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaDescriptorConverter.java
new file mode 100644
index 0000000..15f985d
--- /dev/null
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaDescriptorConverter.java
@@ -0,0 +1,140 @@
+/*
+ * 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++)
+            valCols[i] = convert(valColsCfg.get(i));
+
+        return new SchemaDescriptor(tblId, schemaVer, keyCols, affCols, valCols);
+    }
+}
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaTableValidatorImpl.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaTableValidatorImpl.java
new file mode 100644
index 0000000..622440f
--- /dev/null
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/SchemaTableValidatorImpl.java
@@ -0,0 +1,66 @@
+/*
+ * 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 {
+                SchemaTableImpl tbl = SchemaConfigurationConverter.convert(view);
+
+                Collection<Column> allColumns = new ArrayList<>(tbl.keyColumns());
+                allColumns.addAll(tbl.valueColumns());
+
+                SchemaTableBuilderImpl.validateIndices(tbl.indices(), allColumns);
+            }
+            catch (IllegalArgumentException e) {
+                ctx.addIssue(new ValidationIssue("Validator works success by key " + ctx.currentKey() + ". Found "
+                    + view.columns().size() + " columns"));
+            }
+        }
+
+    }
+
+    /** Private constructor. */
+    private SchemaTableValidatorImpl() {
+    }
+}
diff --git a/modules/schema/src/test/java/org/apache/ignite/internal/schema/builder/HashIndexBuilderTest.java b/modules/schema/src/test/java/org/apache/ignite/internal/schema/builder/HashIndexBuilderTest.java
new file mode 100644
index 0000000..ffb74d0
--- /dev/null
+++ b/modules/schema/src/test/java/org/apache/ignite/internal/schema/builder/HashIndexBuilderTest.java
@@ -0,0 +1,57 @@
+/*
+ * 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 {
+    /**
+     * Build index and check its parameters.
+     */
+    @Test
+    public void testBuild() {
+        HashIndexBuilder builder = SchemaBuilders.hashIndex("testHI")
+            .withColumns("A", "B", "C")
+            .withHints(Collections.singletonMap("param","value"));
+        HashIndex idx = builder.build();
+
+        assertEquals("testHI", idx.name());
+        assertEquals(3, idx.columns().size());
+    }
+
+    /**
+     * Try to create index without columns and check error.
+     */
+    @Test
+    public void testBuildNoColumns() {
+        HashIndexBuilder builder = SchemaBuilders.hashIndex("testHI");
+
+        assertThrows(AssertionError.class, builder::build);
+    }
+}
diff --git a/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/ColumnConfigurationSchema.java b/modules/schema/src/test/java/org/apache/ignite/internal/schema/builder/PartialIndexBuilderTest.java
similarity index 50%
copy from modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/ColumnConfigurationSchema.java
copy to modules/schema/src/test/java/org/apache/ignite/internal/schema/builder/PartialIndexBuilderTest.java
index 69fc03a..50c818c 100644
--- a/modules/api/src/main/java/org/apache/ignite/configuration/schemas/table/ColumnConfigurationSchema.java
+++ b/modules/schema/src/test/java/org/apache/ignite/internal/schema/builder/PartialIndexBuilderTest.java
@@ -15,30 +15,32 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.configuration.schemas.table;
+package org.apache.ignite.internal.schema.builder;
 
-import org.apache.ignite.configuration.annotation.Config;
-import org.apache.ignite.configuration.annotation.ConfigValue;
-import org.apache.ignite.configuration.annotation.Value;
-import org.apache.ignite.configuration.validation.Immutable;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.SchemaBuilders;
+import org.apache.ignite.schema.builder.PartialIndexBuilder;
+import org.junit.jupiter.api.Test;
 
-/** Configuration for single column in SQL table. */
-@Config
-public class ColumnConfigurationSchema {
-    /** Column name. */
-    @Value
-    @Immutable
-    String name;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
-    /** Column type. */
-    @ConfigValue
-    ColumnTypeConfigurationSchema type;
+/**
+ * Tests for partial index builder.
+ */
+public class PartialIndexBuilderTest {
+    /**
+     * Test partial index parameters.
+     */
+    @Test
+    public void testPartialIndexCreate() {
+        PartialIndexBuilder builder = SchemaBuilders.partialIndex("TEST");
+
+        builder.addIndexColumn("A").done();
+        builder.withExpression("WHERE A > 0");
 
-    /** Nullable flag. */
-    @Value
-    boolean nullable;
+        PartialIndex idx = builder.build();
 
-    /** Default value. */
-    @Value(hasDefault = true)
-    String defaultValue = "";
+        assertEquals(1, idx.columns().size());
+        assertEquals("WHERE A > 0", idx.expr());
+    }
 }
diff --git a/modules/schema/src/test/java/org/apache/ignite/internal/schema/builder/PrimaryKeyBuilderTest.java b/modules/schema/src/test/java/org/apache/ignite/internal/schema/builder/PrimaryKeyBuilderTest.java
new file mode 100644
index 0000000..90f3cd3
--- /dev/null
+++ b/modules/schema/src/test/java/org/apache/ignite/internal/schema/builder/PrimaryKeyBuilderTest.java
@@ -0,0 +1,47 @@
+/*
+ * 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.PrimaryIndex;
+import org.apache.ignite.schema.SchemaBuilders;
+import org.apache.ignite.schema.builder.PrimaryIndexBuilder;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Primary key builder test.
+ */
+public class PrimaryKeyBuilderTest {
+    /** Test primary index parameters. */
+    @Test
+    public void testPrimaryKey() {
+        PrimaryIndexBuilder builder = SchemaBuilders.pkIndex();
+
+        builder.addIndexColumn("A").desc().done();
+        builder.addIndexColumn("B").asc().done();
+
+        PrimaryIndex idx = builder.build();
+
+        assertEquals(2, idx.columns().size());
+        assertFalse(idx.columns().get(0).asc());
+        assertTrue(idx.columns().get(1).asc());
+    }
+}
diff --git a/modules/schema/src/test/java/org/apache/ignite/internal/schema/builder/SchemaTableBuilderTest.java b/modules/schema/src/test/java/org/apache/ignite/internal/schema/builder/SchemaTableBuilderTest.java
new file mode 100644
index 0000000..3e0f53a
--- /dev/null
+++ b/modules/schema/src/test/java/org/apache/ignite/internal/schema/builder/SchemaTableBuilderTest.java
@@ -0,0 +1,49 @@
+/*
+ * 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.ColumnType;
+import org.apache.ignite.schema.SchemaBuilders;
+import org.apache.ignite.schema.SchemaTable;
+import org.apache.ignite.schema.builder.SchemaTableBuilder;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+/**
+ * Tests for schema table builder.
+ */
+public class SchemaTableBuilderTest {
+    /**
+     * Create schema table and check its parameters.
+     */
+    @Test
+    public void testBuild() {
+        SchemaTableBuilder builder = SchemaBuilders.tableBuilder("SNAME","TNAME")
+            .columns(
+                SchemaBuilders.column("COL1", ColumnType.DOUBLE).build(),
+                SchemaBuilders.column("COL2", ColumnType.DOUBLE).build())
+            .withPrimaryKey("COL1");
+
+        SchemaTable tbl = builder.build();
+
+        assertEquals("SNAME.TNAME", tbl.canonicalName());
+        assertEquals("TNAME", tbl.name());
+        assertEquals(1, tbl.keyColumns().size());
+    }
+}
diff --git a/modules/schema/src/test/java/org/apache/ignite/internal/schema/builder/SortedIndexBuilderTest.java b/modules/schema/src/test/java/org/apache/ignite/internal/schema/builder/SortedIndexBuilderTest.java
new file mode 100644
index 0000000..36b67e0
--- /dev/null
+++ b/modules/schema/src/test/java/org/apache/ignite/internal/schema/builder/SortedIndexBuilderTest.java
@@ -0,0 +1,49 @@
+/*
+ * 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.SchemaBuilders;
+import org.apache.ignite.schema.SortedIndex;
+import org.apache.ignite.schema.builder.SortedIndexBuilder;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Tests for sorted index builder.
+ */
+public class SortedIndexBuilderTest {
+    /**
+     * Build sorted index and check it's parameters.
+     */
+    @Test
+    public void testBuild() {
+        SortedIndexBuilder builder = SchemaBuilders.sortedIndex("SIDX");
+
+        builder.addIndexColumn("A").asc().done();
+        builder.addIndexColumn("B").desc().done();
+
+        builder.unique();
+
+        SortedIndex idx = builder.build();
+
+        assertTrue(idx.unique());
+        assertEquals(2, idx.indexedColumns().size());
+    }
+}
diff --git a/modules/schema/src/test/java/org/apache/ignite/internal/schema/builder/TableColumnBuilderTest.java b/modules/schema/src/test/java/org/apache/ignite/internal/schema/builder/TableColumnBuilderTest.java
new file mode 100644
index 0000000..c0f7af3
--- /dev/null
+++ b/modules/schema/src/test/java/org/apache/ignite/internal/schema/builder/TableColumnBuilderTest.java
@@ -0,0 +1,47 @@
+/*
+ * 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.Column;
+import org.apache.ignite.schema.ColumnType;
+import org.apache.ignite.schema.SchemaBuilders;
+import org.apache.ignite.schema.builder.TableColumnBuilder;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+
+/**
+ * Tests for table column builder.
+ */
+public class TableColumnBuilderTest {
+    /**
+     * Check column parameters.
+     */
+    @Test
+    public void testCreateColumn() {
+        TableColumnBuilder builder = SchemaBuilders.column("TEST", ColumnType.DOUBLE);
+
+        Column col = builder.asNonNull().withDefaultValue(1.).build();
+
+        assertEquals("TEST", col.name());
+        assertEquals(ColumnType.DOUBLE, col.type());
+        assertEquals(1., col.defaultValue());
+        assertFalse(col.nullable());
+    }
+}
diff --git a/modules/schema/src/test/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverterTest.java b/modules/schema/src/test/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverterTest.java
new file mode 100644
index 0000000..bb78ef2
--- /dev/null
+++ b/modules/schema/src/test/java/org/apache/ignite/internal/schema/configuration/SchemaConfigurationConverterTest.java
@@ -0,0 +1,224 @@
+/*
+ * 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 java.util.Collection;
+import java.util.Collections;
+import java.util.concurrent.ExecutionException;
+import java.util.stream.Collectors;
+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.schema.ColumnType;
+import org.apache.ignite.schema.HashIndex;
+import org.apache.ignite.schema.PartialIndex;
+import org.apache.ignite.schema.PrimaryIndex;
+import org.apache.ignite.schema.SchemaBuilders;
+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 org.apache.ignite.schema.builder.HashIndexBuilder;
+import org.apache.ignite.schema.builder.PartialIndexBuilder;
+import org.apache.ignite.schema.builder.PrimaryIndexBuilder;
+import org.apache.ignite.schema.builder.SchemaTableBuilder;
+import org.apache.ignite.schema.builder.SortedIndexBuilder;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+
+/**
+ * SchemaConfigurationConverter tests.
+ */
+@SuppressWarnings("InstanceVariableMayNotBeInitialized")
+public class SchemaConfigurationConverterTest {
+    /** Table builder. */
+    private SchemaTableBuilder tblBuilder;
+
+    /** Configuration registry with one table for each test. */
+    private ConfigurationRegistry confRegistry;
+
+    /**
+     * Prepare configuration registry for test.
+     *
+     * @throws ExecutionException If failed.
+     * @throws InterruptedException If failed.
+     */
+    @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()));
+
+        tblBuilder = SchemaBuilders.tableBuilder("SNAME","TNAME")
+            .columns(
+                SchemaBuilders.column("COL1", ColumnType.DOUBLE).build(),
+                SchemaBuilders.column("COL2", ColumnType.DOUBLE).build(),
+                SchemaBuilders.column("A", ColumnType.INT8).build(),
+                SchemaBuilders.column("B", ColumnType.INT8).build(),
+                SchemaBuilders.column("C", ColumnType.INT8).build()
+            ).withPrimaryKey("COL1");
+
+        SchemaTable tbl = tblBuilder.build();
+
+        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 {
+        HashIndexBuilder builder = SchemaBuilders.hashIndex("testHI")
+            .withColumns("A", "B", "C")
+            .withHints(Collections.singletonMap("param","value"));
+        HashIndex idx = builder.build();
+
+        getTbl().change(ch -> SchemaConfigurationConverter.addIndex(idx, ch)).get();
+
+        SchemaTable tbl = SchemaConfigurationConverter.convert(getTbl().value());
+
+        HashIndex idx2 = (HashIndex)getIdx(idx.name(), tbl.indices());
+
+        assertNotNull(idx2);
+        assertEquals("HASH", idx2.type());
+        assertEquals(3, idx2.columns().size());
+    }
+
+    /**
+     * Add/remove SortedIndex into configuration and read it back.
+     */
+    @Test
+    public void testConvertSortedIndex() throws ExecutionException, InterruptedException {
+        SortedIndexBuilder builder = SchemaBuilders.sortedIndex("SIDX");
+
+        builder.addIndexColumn("A").asc().done();
+        builder.addIndexColumn("B").desc().done();
+
+        builder.unique();
+
+        SortedIndex idx = builder.build();
+
+        getTbl().change(ch -> SchemaConfigurationConverter.addIndex(idx, ch)).get();
+
+        SchemaTable tbl = SchemaConfigurationConverter.convert(getTbl().value());
+
+        SortedIndex idx2 = (SortedIndex)getIdx(idx.name(), tbl.indices());
+
+        assertNotNull(idx2);
+        assertEquals("SORTED", idx2.type());
+        assertEquals(2, idx2.columns().size());
+    }
+
+    /**
+     * Add/remove PrimaryIndex into configuration and read it back.
+     */
+    @Test
+    public void testPKIndex() throws ExecutionException, InterruptedException {
+        PrimaryIndexBuilder builder = SchemaBuilders.pkIndex();
+        builder.addIndexColumn("COL1").desc().done();
+        builder.addIndexColumn("A").desc().done();
+        builder.addIndexColumn("B").asc().done();
+        builder.withAffinityColumns("COL1");
+
+        PrimaryIndex idx = builder.build();
+
+        getTbl().change(ch -> SchemaConfigurationConverter.addIndex(idx, ch)).get();
+
+        SchemaTable tbl = SchemaConfigurationConverter.convert(getTbl().value());
+
+        PrimaryIndex idx2 = (PrimaryIndex)getIdx(idx.name(), tbl.indices());
+
+        assertNotNull(idx2);
+        assertEquals("PK", idx2.type());
+        assertEquals(idx.columns().stream().map(SortedIndexColumn::name).collect(Collectors.toList()),
+            idx2.columns().stream().map(SortedIndexColumn::name).collect(Collectors.toList()));
+        assertEquals(idx.affinityColumns(), idx2.affinityColumns());
+    }
+
+    /**
+     * Add/remove PartialIndex into configuration and read it back.
+     */
+    @Test
+    public void testPartialIndex() throws ExecutionException, InterruptedException {
+        PartialIndexBuilder builder = SchemaBuilders.partialIndex("TEST");
+
+        builder.addIndexColumn("A").done();
+        builder.withExpression("WHERE A > 0");
+
+        PartialIndex idx = builder.build();
+
+        getTbl().change(ch -> SchemaConfigurationConverter.addIndex(idx, ch)).get();
+
+        SchemaTable tbl = SchemaConfigurationConverter.convert(getTbl().value());
+
+        PartialIndex idx2 = (PartialIndex) getIdx(idx.name(), tbl.indices());
+
+        assertNotNull(idx2);
+        assertEquals("PARTIAL", idx2.type());
+        assertEquals(idx.columns().size(), idx2.columns().size());
+    }
+
+    /**
+     * Add/remove table and read it back.
+     */
+    @Test
+    public void testConvertTable() {
+        SchemaTable tbl = tblBuilder.build();
+
+        TableConfiguration tblCfg = confRegistry.getConfiguration(TablesConfiguration.KEY).tables()
+            .get(tbl.canonicalName());
+
+        SchemaTable tbl2 = SchemaConfigurationConverter.convert(tblCfg);
+
+        assertEquals(tbl.canonicalName(), tbl2.canonicalName());
+        assertEquals(tbl.indices().size(), tbl2.indices().size());
+        assertEquals(tbl.keyColumns().size(), tbl2.keyColumns().size());
+        assertEquals(tbl.affinityColumns().size(), tbl2.affinityColumns().size());
+        assertEquals(tbl.valueColumns().size(), tbl2.valueColumns().size());
+    }
+
+    /**
+     * Get tests default table configuration.
+     *
+     * @return Configuration of default table.
+     */
+    private TableConfiguration getTbl() {
+        return confRegistry.getConfiguration(TablesConfiguration.KEY).tables().get(tblBuilder.build().canonicalName());
+    }
+
+    /**
+     * Get table index by name.
+     *
+     * @param name Index name to find.
+     * @param idxs Table indexes.
+     * @return Index or {@code null} if there are no index with such name.
+     */
+    private TableIndex getIdx(String name, Collection<TableIndex> idxs) {
+        return idxs.stream().filter(idx -> name.equals(idx.name())).findAny().orElse(null);
+    }
+}
diff --git a/modules/schema/src/test/java/org/apache/ignite/internal/schema/configuration/SchemaDescriptorConverterTest.java b/modules/schema/src/test/java/org/apache/ignite/internal/schema/configuration/SchemaDescriptorConverterTest.java
new file mode 100644
index 0000000..4dfc6d6
--- /dev/null
+++ b/modules/schema/src/test/java/org/apache/ignite/internal/schema/configuration/SchemaDescriptorConverterTest.java
@@ -0,0 +1,190 @@
+/*
+ * 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, tblDscr.keyColumns().length());
+        assertEquals(2, tblDscr.affinityColumns().length);
+        assertEquals(columns - 2, tblDscr.valueColumns().length());
+    }
+
+    /**
+     * Convert table with complex primary key with affinity column configured and check it.
+     */
+    @Test
+    public void testComplexPrimaryIndexWithAffinity() {
+        SchemaTableBuilder bldr = getBuilder(false, false);
+        SchemaTable tblSchm = bldr.withIndex(SchemaBuilders.pkIndex()
+            .addIndexColumn("INT8").done()
+            .addIndexColumn("ID").done()
+            .withAffinityColumns("INT8")
+            .build()
+        ).build();
+
+        SchemaDescriptor tblDscr = SchemaDescriptorConverter.convert(UUID.randomUUID(), 1, tblSchm);
+
+        assertEquals(2, tblDscr.keyColumns().length());
+        assertEquals(1, tblDscr.affinityColumns().length);
+        assertEquals(columns - 2, tblDscr.valueColumns().length());
+    }
+
+    /**
+     * 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: IGNITE-13750 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);
+    }
+}
diff --git a/modules/schema/src/test/java/org/apache/ignite/internal/schema/configuration/TestConfigurationStorage.java b/modules/schema/src/test/java/org/apache/ignite/internal/schema/configuration/TestConfigurationStorage.java
new file mode 100644
index 0000000..7b8f697
--- /dev/null
+++ b/modules/schema/src/test/java/org/apache/ignite/internal/schema/configuration/TestConfigurationStorage.java
@@ -0,0 +1,69 @@
+/*
+ * 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 java.io.Serializable;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import org.apache.ignite.configuration.storage.ConfigurationStorage;
+import org.apache.ignite.configuration.storage.ConfigurationStorageListener;
+import org.apache.ignite.configuration.storage.ConfigurationType;
+import org.apache.ignite.configuration.storage.Data;
+import org.apache.ignite.configuration.storage.StorageException;
+
+/**
+ * Configuration storage for tests.
+ */
+public class TestConfigurationStorage implements ConfigurationStorage {
+    /** */
+    private final Set<ConfigurationStorageListener> listeners = new HashSet<>();
+
+    /** {@inheritDoc} */
+    @Override public Data readAll() throws StorageException {
+        return new Data(Collections.emptyMap(), 0);
+    }
+
+    /** {@inheritDoc} */
+    @Override public CompletableFuture<Boolean> write(Map<String, Serializable> newValues, long version) {
+        for (ConfigurationStorageListener listener : listeners)
+            listener.onEntriesChanged(new Data(newValues, version + 1));
+
+        return CompletableFuture.completedFuture(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void addListener(ConfigurationStorageListener listener) {
+        listeners.add(listener);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void removeListener(ConfigurationStorageListener listener) {
+        listeners.remove(listener);
+    }
+
+    @Override public void notifyApplied(long storageRevision) {
+    }
+
+    /** {@inheritDoc} */
+    @Override public ConfigurationType type() {
+        return ConfigurationType.DISTRIBUTED;
+    }
+}