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/11/11 12:55:54 UTC

[ignite-3] 01/01: Drop live schema support.

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

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

commit c76139bc3b3c6a3dbb0175dea8a2a496cdeb1202
Author: Andrew Mashenkov <an...@gmail.com>
AuthorDate: Thu Nov 11 15:55:46 2021 +0300

    Drop live schema support.
---
 .../schema/definition/SchemaManagementMode.java    |  37 ---
 .../ignite/internal/client/table/ClientTuple.java  |   1 -
 .../ignite/client/fakes/FakeInternalTable.java     |  13 -
 .../runner/app/ItLiveSchemaChangeKvViewTest.java   | 214 --------------
 .../runner/app/ItLiveSchemaChangeTableTest.java    | 320 ---------------------
 .../runner/app/ItSchemaChangeKvViewTest.java       |  15 +
 .../runner/app/ItSchemaChangeTableViewTest.java    | 141 +++++----
 modules/schema/README.md                           |  17 +-
 .../schema/marshaller/TupleMarshallerImpl.java     | 136 ++-------
 .../ignite/internal/table/InternalTable.java       |  13 -
 .../internal/table/KeyValueBinaryViewImpl.java     |   2 +-
 .../internal/table/RecordBinaryViewImpl.java       |   2 +-
 .../apache/ignite/internal/table/TableImpl.java    |  10 -
 .../distributed/storage/InternalTableImpl.java     |  18 --
 .../TupleMarshallerFixlenOnlyBenchmark.java        |   2 +-
 .../TupleMarshallerVarlenOnlyBenchmark.java        |   2 +-
 .../internal/table/MutableRowTupleAdapterTest.java |  25 +-
 ...erationsTest.java => SchemaValidationTest.java} |  12 +-
 .../table/impl/DummyInternalTableImpl.java         |  12 -
 .../table/type/NumericTypesSerializerTest.java     |  21 +-
 20 files changed, 146 insertions(+), 867 deletions(-)

diff --git a/modules/api/src/main/java/org/apache/ignite/schema/definition/SchemaManagementMode.java b/modules/api/src/main/java/org/apache/ignite/schema/definition/SchemaManagementMode.java
deleted file mode 100644
index 40049cd..0000000
--- a/modules/api/src/main/java/org/apache/ignite/schema/definition/SchemaManagementMode.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.schema.definition;
-
-/**
- * Schema mode.
- *
- * <p>Defines the way inserting data will be validated against the schema and schema evolution capabilities.
- */
-//TODO: rename to MANUAL and AUTO?
-public enum SchemaManagementMode {
-    /**
-     * Normal mode offers strong validation for the inserting data. Explicit schema changes only are allowed.
-     */
-    STRICT,
-
-    /**
-     * Extended mode that allows the schema to be fit the inserting data automatically. Only safe implicit schema changes are allowed, e.g.
-     * adding extra columns and widening column type. Changes like column removal or narrowing column type won't be applied implicitly.
-     */
-    LIVE
-}
diff --git a/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientTuple.java b/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientTuple.java
index 49dc5bb..f63e4eb 100644
--- a/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientTuple.java
+++ b/modules/client/src/main/java/org/apache/ignite/internal/client/table/ClientTuple.java
@@ -78,7 +78,6 @@ public final class ClientTuple implements Tuple {
     /** {@inheritDoc} */
     @Override
     public Tuple set(@NotNull String columnName, Object value) {
-        // TODO: Live schema and schema evolution support IGNITE-15194
         var col = schema.column(columnName);
 
         vals[col.schemaIndex() - minColumnIndex] = value == null ? NULL_OBJ : value;
diff --git a/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeInternalTable.java b/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeInternalTable.java
index a04546d..9946934 100644
--- a/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeInternalTable.java
+++ b/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeInternalTable.java
@@ -30,7 +30,6 @@ import org.apache.ignite.internal.storage.engine.TableStorage;
 import org.apache.ignite.internal.table.InternalTable;
 import org.apache.ignite.lang.IgniteInternalException;
 import org.apache.ignite.lang.IgniteUuid;
-import org.apache.ignite.schema.definition.SchemaManagementMode;
 import org.apache.ignite.tx.Transaction;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
@@ -85,18 +84,6 @@ public class FakeInternalTable implements InternalTable {
     
     /** {@inheritDoc} */
     @Override
-    public @NotNull SchemaManagementMode schemaMode() {
-        return SchemaManagementMode.STRICT;
-    }
-    
-    /** {@inheritDoc} */
-    @Override
-    public void schema(SchemaManagementMode schemaMode) {
-        // No-op.
-    }
-    
-    /** {@inheritDoc} */
-    @Override
     public CompletableFuture<BinaryRow> get(BinaryRow keyRow, @Nullable Transaction tx) {
         return CompletableFuture.completedFuture(data.get(keyRow.keySlice()));
     }
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItLiveSchemaChangeKvViewTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItLiveSchemaChangeKvViewTest.java
deleted file mode 100644
index 3016ed1..0000000
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItLiveSchemaChangeKvViewTest.java
+++ /dev/null
@@ -1,214 +0,0 @@
-/*
- * 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.runner.app;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.util.List;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.internal.schema.SchemaDescriptor;
-import org.apache.ignite.internal.schema.SchemaMismatchException;
-import org.apache.ignite.internal.table.TableImpl;
-import org.apache.ignite.schema.definition.SchemaManagementMode;
-import org.apache.ignite.table.KeyValueView;
-import org.apache.ignite.table.Table;
-import org.apache.ignite.table.Tuple;
-import org.junit.jupiter.api.Test;
-
-/**
- * Live schema tests for KV View.
- */
-class ItLiveSchemaChangeKvViewTest extends AbstractSchemaChangeTest {
-    /**
-     * Check an operation failed if an unknown column found in case of STRICT_SCHEMA mode is on.
-     */
-    @Test
-    public void testStrictSchemaInsertRowOfNewSchema() {
-        List<Ignite> grid = startGrid();
-        
-        createTable(grid);
-        
-        KeyValueView<Tuple, Tuple> view = grid.get(0).tables().table(TABLE).keyValueView();
-        
-        assertThrowsWithCause(SchemaMismatchException.class,
-                () -> view.put(Tuple.create().set("key", 1L), Tuple.create().set("unknownColumn", 10)));
-    }
-    
-    /**
-     * Check live schema kvBinaryView add columns.
-     */
-    @Test
-    public void testLiveSchemaAddColumns() {
-        List<Ignite> grid = startGrid();
-        
-        createTable(grid);
-        
-        Table tbl = grid.get(1).tables().table(TABLE);
-        
-        ((TableImpl) tbl).schemaMode(SchemaManagementMode.LIVE);
-        
-        KeyValueView<Tuple, Tuple> kvBinaryView = tbl.keyValueView();
-        
-        Tuple key = Tuple.create().set("key", 1L);
-        Tuple val = Tuple.create().set("valStrNew", "111").set("valIntNew", 333);
-        
-        kvBinaryView.put(key, val);
-        
-        Tuple res = kvBinaryView.get(key);
-        assertEquals("111", res.value("valStrNew"));
-        assertEquals(Integer.valueOf(333), res.value("valIntNew"));
-    }
-    
-    /**
-     * Check strict schema works correctly after live schema.
-     */
-    @Test
-    public void testLiveSchemaAddColumnsSwitchToStrict() {
-        List<Ignite> grid = startGrid();
-        
-        createTable(grid);
-        
-        Table tbl = grid.get(0).tables().table(TABLE);
-        
-        ((TableImpl) tbl).schemaMode(SchemaManagementMode.LIVE);
-        
-        KeyValueView<Tuple, Tuple> kvBinaryView = tbl.keyValueView();
-        
-        Tuple key = Tuple.create().set("key", 1L);
-        Tuple val = Tuple.create().set("valStrNew", "111").set("valIntNew", 333);
-        
-        kvBinaryView.put(key, val);
-        
-        Tuple res = kvBinaryView.get(key);
-        assertEquals("111", res.value("valStrNew"));
-        assertEquals(Integer.valueOf(333), res.value("valIntNew"));
-        
-        ((TableImpl) tbl).schemaMode(SchemaManagementMode.STRICT);
-        
-        Tuple anotherKey = Tuple.create().set("key", 2L);
-        Tuple anotherVal = Tuple.create().set("valStrNew", "111").set("valIntNew", 333);
-        
-        kvBinaryView.put(anotherKey, anotherVal);
-        
-        Tuple newRes = kvBinaryView.get(anotherKey);
-        
-        assertEquals("111", newRes.value("valStrNew"));
-        assertEquals(Integer.valueOf(333), newRes.value("valIntNew"));
-        
-        assertThrowsWithCause(SchemaMismatchException.class,
-                () -> kvBinaryView.put(Tuple.create().set("key", 1L), Tuple.create().set("unknownColumn", 10)));
-    }
-    
-    /**
-     * Check upsert row of old schema with row of new schema.
-     */
-    @Test
-    public void testLiveSchemaUpsertOldSchemaRow() {
-        List<Ignite> grid = startGrid();
-        
-        createTable(grid);
-        
-        Table tbl = grid.get(1).tables().table(TABLE);
-        
-        ((TableImpl) tbl).schemaMode(SchemaManagementMode.LIVE);
-        
-        KeyValueView<Tuple, Tuple> view = tbl.keyValueView();
-        
-        Tuple oldSchemaKey = Tuple.create().set("key", 32L);
-        Tuple oldSchemaVal = Tuple.create().set("valInt", 111).set("valStr", "str");
-        
-        view.put(oldSchemaKey, oldSchemaVal);
-        
-        Tuple upsertOldSchemaVal = Tuple.create().set("valStrNew", "111").set("valIntNew", 333);
-        
-        view.put(oldSchemaKey, upsertOldSchemaVal);
-        
-        Tuple oldSchemaRes = view.get(oldSchemaKey);
-        
-        assertEquals("111", oldSchemaRes.value("valStrNew"));
-        assertEquals(Integer.valueOf(333), oldSchemaRes.value("valIntNew"));
-    }
-    
-    /**
-     * Check upsert row of old schema with row of new schema.
-     */
-    @Test
-    public void testLiveSchemaUpsertSchemaTwice() {
-        List<Ignite> grid = startGrid();
-        
-        createTable(grid);
-        
-        Table tbl = grid.get(1).tables().table(TABLE);
-        
-        ((TableImpl) tbl).schemaMode(SchemaManagementMode.LIVE);
-        
-        KeyValueView<Tuple, Tuple> view = tbl.keyValueView();
-        
-        Tuple oldSchemaKey = Tuple.create().set("key", 32L);
-        
-        Tuple oldSchemaVal = Tuple.create().set("valInt", 111).set("valStr", "str");
-        Tuple upsertOldSchemaVal = Tuple.create().set("valStrNew", "111").set("valIntNew", 333);
-        Tuple secondUpsertOldSchemaVal = Tuple.create().set("valStrNew", "111").set("valIntNew", 333).set("anotherNewVal", 48L);
-        
-        view.put(oldSchemaKey, oldSchemaVal);
-        view.put(oldSchemaKey, upsertOldSchemaVal);
-        view.put(oldSchemaKey, secondUpsertOldSchemaVal);
-        
-        Tuple oldSchemaRes = view.get(oldSchemaKey);
-        
-        assertEquals("111", oldSchemaRes.value("valStrNew"));
-        assertEquals(Integer.valueOf(333), oldSchemaRes.value("valIntNew"));
-        assertEquals(Long.valueOf(48L), oldSchemaRes.value("anotherNewVal"));
-    }
-    
-    /**
-     * Check inserting row of old schema will not lead to column removal.
-     */
-    @Test
-    public void testLiveSchemaInsertOldSchemaRow() {
-        List<Ignite> grid = startGrid();
-        
-        createTable(grid);
-        
-        Table tbl = grid.get(0).tables().table(TABLE);
-        
-        ((TableImpl) tbl).schemaMode(SchemaManagementMode.LIVE);
-        
-        KeyValueView<Tuple, Tuple> view = tbl.keyValueView();
-        
-        Tuple oldSchemaKey = Tuple.create().set("key", 32L);
-        Tuple oldSchemaVal = Tuple.create().set("valInt", 111).set("valStr", "str");
-        
-        Tuple newSchemaKey = Tuple.create().set("key", 1L);
-        Tuple newSchemaVal = Tuple.create().set("valStrNew", "111").set("valIntNew", 333);
-        
-        view.put(newSchemaKey, newSchemaVal);
-        view.put(oldSchemaKey, oldSchemaVal);
-        
-        Tuple res = view.get(newSchemaKey);
-        
-        assertEquals("111", res.value("valStrNew"));
-        assertEquals(Integer.valueOf(333), res.value("valIntNew"));
-        
-        SchemaDescriptor schema = ((TableImpl) tbl).schemaView().schema();
-        
-        assertTrue(schema.columnNames().contains("valStrNew"));
-        assertTrue(schema.columnNames().contains("valIntNew"));
-    }
-}
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItLiveSchemaChangeTableTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItLiveSchemaChangeTableTest.java
deleted file mode 100644
index ff58be3..0000000
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItLiveSchemaChangeTableTest.java
+++ /dev/null
@@ -1,320 +0,0 @@
-/*
- * 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.runner.app;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNull;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-import java.util.List;
-import java.util.UUID;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.internal.schema.InvalidTypeException;
-import org.apache.ignite.internal.schema.SchemaDescriptor;
-import org.apache.ignite.internal.schema.SchemaMismatchException;
-import org.apache.ignite.internal.table.TableImpl;
-import org.apache.ignite.schema.definition.SchemaManagementMode;
-import org.apache.ignite.table.RecordView;
-import org.apache.ignite.table.Table;
-import org.apache.ignite.table.Tuple;
-import org.junit.jupiter.api.Test;
-
-/**
- * Live schema tests.
- */
-class ItLiveSchemaChangeTableTest extends AbstractSchemaChangeTest {
-    /**
-     * Check exception for unknown column when STRICT_SCHEMA is enabled.
-     */
-    @Test
-    public void testStrictSchemaInsertRowOfNewSchema() {
-        List<Ignite> grid = startGrid();
-        
-        createTable(grid);
-        
-        Table tbl = grid.get(0).tables().table(TABLE);
-        
-        Tuple tuple = Tuple.create().set("key", 1L).set("unknownColumn", 10);
-        
-        assertThrowsWithCause(SchemaMismatchException.class, () -> tbl.recordView().insert(tuple));
-    }
-    
-    /**
-     * Check insert row of new schema.
-     */
-    @Test
-    public void testLiveSchemaInsertRowOfNewSchema() {
-        List<Ignite> grid = startGrid();
-        
-        createTable(grid);
-        
-        Table tbl = grid.get(0).tables().table(TABLE);
-        RecordView<Tuple> recView = tbl.recordView();
-        
-        ((TableImpl) tbl).schemaMode(SchemaManagementMode.LIVE);
-        
-        Tuple row = Tuple.create().set("key", 1L).set("valStrNew", "111").set("valIntNew", 333);
-        
-        recView.insert(row);
-        
-        Tuple res = recView.get(Tuple.create().set("key", 1L));
-        
-        assertEquals("111", res.value("valStrNew"));
-        assertEquals(Integer.valueOf(333), res.value("valIntNew"));
-    }
-    
-    /**
-     * Check upsert row of old schema with row of new schema.
-     */
-    @Test
-    public void testLiveSchemaUpsertOldSchemaRow() {
-        List<Ignite> grid = startGrid();
-        
-        createTable(grid);
-        
-        Table tbl = grid.get(0).tables().table(TABLE);
-        RecordView<Tuple> recView = tbl.recordView();
-        
-        Tuple oldSchemaTuple = Tuple.create().set("key", 32L).set("valInt", 111).set("valStr", "str");
-        
-        recView.insert(oldSchemaTuple);
-        
-        ((TableImpl) tbl).schemaMode(SchemaManagementMode.LIVE);
-        
-        Tuple upsertOldSchemaTuple = Tuple.create().set("key", 32L).set("valStrNew", "111").set("valIntNew", 333);
-        
-        recView.upsert(upsertOldSchemaTuple);
-        
-        Tuple oldSchemaRes = recView.get(Tuple.create().set("key", 32L));
-        
-        assertEquals("111", oldSchemaRes.value("valStrNew"));
-        assertEquals(Integer.valueOf(333), oldSchemaRes.value("valIntNew"));
-    }
-    
-    /**
-     * Check inserting row of old schema will not lead to column removal.
-     */
-    @Test
-    public void testLiveSchemaInsertOldSchemaRow() {
-        List<Ignite> grid = startGrid();
-        
-        createTable(grid);
-        
-        Table tbl = grid.get(0).tables().table(TABLE);
-        RecordView<Tuple> recView = tbl.recordView();
-        
-        Tuple oldSchemaTuple = Tuple.create().set("key", 32L).set("valInt", 111).set("valStr", "str");
-        
-        ((TableImpl) tbl).schemaMode(SchemaManagementMode.LIVE);
-        
-        Tuple row = Tuple.create().set("key", 1L).set("valStrNew", "111").set("valIntNew", 333);
-        
-        recView.insert(row);
-        recView.insert(oldSchemaTuple);
-        
-        Tuple res = recView.get(Tuple.create().set("key", 1L));
-        
-        assertEquals("111", res.value("valStrNew"));
-        assertEquals(Integer.valueOf(333), res.value("valIntNew"));
-        
-        SchemaDescriptor schema = ((TableImpl) tbl).schemaView().schema();
-        
-        assertTrue(schema.columnNames().contains("valStrNew"));
-        assertTrue(schema.columnNames().contains("valIntNew"));
-    }
-    
-    /**
-     * Check strict schema works correctly after live schema.
-     */
-    @Test
-    public void testLiveSchemaAddColumnsSwitchToStrict() {
-        List<Ignite> grid = startGrid();
-        
-        createTable(grid);
-        
-        Table tbl = grid.get(0).tables().table(TABLE);
-        RecordView<Tuple> recView = tbl.recordView();
-        
-        ((TableImpl) tbl).schemaMode(SchemaManagementMode.LIVE);
-        
-        Tuple val = Tuple.create().set("key", 1L).set("valStrNew", "111").set("valIntNew", 333);
-        
-        recView.insert(val);
-        
-        Tuple res = recView.get(Tuple.create().set("key", 1L));
-        assertEquals("111", res.value("valStrNew"));
-        assertEquals(Integer.valueOf(333), res.value("valIntNew"));
-        
-        ((TableImpl) tbl).schemaMode(SchemaManagementMode.STRICT);
-        
-        Tuple anotherKey = Tuple.create().set("key", 2L).set("valStrNew", "111").set("valIntNew", 333);
-        
-        recView.insert(anotherKey);
-        
-        Tuple newRes = recView.get(Tuple.create().set("key", 2L));
-        
-        assertEquals("111", newRes.value("valStrNew"));
-        assertEquals(Integer.valueOf(333), newRes.value("valIntNew"));
-        
-        assertThrowsWithCause(SchemaMismatchException.class, () -> recView.insert(Tuple.create().set("key", 1L).set("unknownColumn", 10)));
-    }
-    
-    /**
-     * Check upsert row of old schema with row of new schema.
-     */
-    @Test
-    public void testLiveSchemaUpsertSchemaTwice() {
-        List<Ignite> grid = startGrid();
-        
-        createTable(grid);
-        
-        Table tbl = grid.get(0).tables().table(TABLE);
-        RecordView<Tuple> recView = tbl.recordView();
-        
-        ((TableImpl) tbl).schemaMode(SchemaManagementMode.LIVE);
-        
-        Tuple oldSchemaVal = Tuple.create().set("key", 32L).set("valInt", 111).set("valStr", "str");
-        Tuple upsertOldSchemaVal = Tuple.create().set("key", 32L).set("valStrNew", "111").set("valIntNew", 333);
-        Tuple secondUpsertOldSchemaVal = Tuple.create().set("key", 32L).set("valStrNew", "111").set("valIntNew", 333)
-                .set("anotherNewVal", 48L);
-        
-        recView.insert(oldSchemaVal);
-        recView.upsert(upsertOldSchemaVal);
-        recView.upsert(secondUpsertOldSchemaVal);
-        
-        Tuple oldSchemaRes = recView.get(Tuple.create().set("key", 32L));
-        
-        assertEquals("111", oldSchemaRes.value("valStrNew"));
-        assertEquals(Integer.valueOf(333), oldSchemaRes.value("valIntNew"));
-        assertEquals(Long.valueOf(48L), oldSchemaRes.value("anotherNewVal"));
-    }
-    
-    /**
-     * Check live schema tuple can handle different value types.
-     */
-    @Test
-    public void testLiveSchemaDifferentColumnTypes() {
-        List<Ignite> grid = startGrid();
-        
-        createTable(grid);
-        
-        Table tbl = grid.get(0).tables().table(TABLE);
-        RecordView<Tuple> recView = tbl.recordView();
-        
-        ((TableImpl) tbl).schemaMode(SchemaManagementMode.LIVE);
-        
-        UUID uuid = UUID.randomUUID();
-        
-        Tuple row = Tuple.create()
-                .set("key", 1L)
-                .set("valByteNew", (byte) 10)
-                .set("valShortNew", (short) 48)
-                .set("valIntNew", 333)
-                .set("valLongNew", 55L)
-                .set("valFloatNew", 32.23f)
-                .set("valDoubleNew", 100.101d)
-                .set("valStrNew", "111")
-                .set("valUUIDNew", uuid);
-        
-        recView.insert(row);
-        
-        Tuple res = recView.get(Tuple.create().set("key", 1L));
-        
-        assertEquals(Byte.valueOf((byte) 10), res.value("valByteNew"));
-        assertEquals(Short.valueOf((short) 48), res.value("valShortNew"));
-        assertEquals(Integer.valueOf(333), res.value("valIntNew"));
-        assertEquals(Long.valueOf(55L), res.value("valLongNew"));
-        assertEquals(32.23f, res.value("valFloatNew"), 0.001f);
-        assertEquals(100.101d, res.value("valDoubleNew"), 0.001f);
-        
-        assertEquals("111", res.value("valStrNew"));
-        assertEquals(uuid, res.value("valUUIDNew"));
-        
-        Tuple secondRow = Tuple.create().set("key", 2L);
-        
-        recView.insert(secondRow);
-        
-        Tuple nullRes = recView.get(secondRow);
-        
-        assertNull(nullRes.value("valByteNew"));
-        assertNull(nullRes.value("valShortNew"));
-        assertNull(nullRes.value("valIntNew"));
-        assertNull(nullRes.value("valLongNew"));
-        assertNull(nullRes.value("valFloatNew"));
-        assertNull(nullRes.value("valDoubleNew"));
-        assertNull(nullRes.value("valUUIDNew"));
-        assertNull(nullRes.value("valStrNew"));
-    }
-    
-    /**
-     * Check live schema tuple update schema only once.
-     */
-    @Test
-    public void testLiveSchemaBuilderUpdateSchemaOnlyOnce() {
-        List<Ignite> grid = startGrid();
-        
-        createTable(grid);
-        
-        Table tbl = grid.get(0).tables().table(TABLE);
-        RecordView<Tuple> recView = tbl.recordView();
-        
-        ((TableImpl) tbl).schemaMode(SchemaManagementMode.LIVE);
-        
-        UUID uuid = UUID.randomUUID();
-        
-        Tuple row = Tuple.create()
-                .set("key", 1L)
-                .set("valByteNew", (byte) 10)
-                .set("valShortNew", (short) 48)
-                .set("valIntNew", 333)
-                .set("valLongNew", 55L)
-                .set("valFloatNew", 32.23f)
-                .set("valDoubleNew", 100.101d)
-                .set("valStrNew", "111")
-                .set("valUUIDNew", uuid);
-        
-        recView.insert(row);
-        
-        SchemaDescriptor schema = ((TableImpl) tbl).schemaView().schema();
-        
-        assertEquals(2, schema.version());
-    }
-    
-    /**
-     * Check live schema tuple can handle unsupported values and null`s correctly.
-     */
-    @Test
-    public void testLiveSchemaNullAndUnsupportedTypes() {
-        List<Ignite> grid = startGrid();
-        
-        createTable(grid);
-        
-        Table tbl = grid.get(0).tables().table(TABLE);
-        RecordView<Tuple> recView = tbl.recordView();
-        
-        ((TableImpl) tbl).schemaMode(SchemaManagementMode.LIVE);
-        
-        Tuple rowWithObject = Tuple.create().set("key", 1L).set("newBrokenColumn", new Object());
-        
-        assertThrowsWithCause(InvalidTypeException.class, () -> recView.insert(rowWithObject));
-        
-        Tuple rowWithNull = Tuple.create().set("key", 1L).set("valStrNew", null).set("valIntNew", 333);
-        
-        assertThrowsWithCause(InvalidTypeException.class, () -> recView.insert(rowWithNull));
-    }
-}
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItSchemaChangeKvViewTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItSchemaChangeKvViewTest.java
index 820d4c0..5b4be6a 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItSchemaChangeKvViewTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItSchemaChangeKvViewTest.java
@@ -278,4 +278,19 @@ class ItSchemaChangeKvViewTest extends AbstractSchemaChangeTest {
         assertEquals("brandNewDefault", kvView.get(keyTuple3).value("valStr"));
         assertEquals("brandNewDefault", kvView.get(keyTuple3).value("val"));
     }
+    
+    /**
+     * Check an operation failed if an unknown column found.
+     */
+    @Test
+    public void testInsertRowOfDifferentSchema() {
+        List<Ignite> grid = startGrid();
+        
+        createTable(grid);
+        
+        KeyValueView<Tuple, Tuple> view = grid.get(0).tables().table(TABLE).keyValueView();
+        
+        assertThrowsWithCause(SchemaMismatchException.class,
+                () -> view.put(Tuple.create().set("key", 1L), Tuple.create().set("unknownColumn", 10)));
+    }
 }
diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItSchemaChangeTableViewTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItSchemaChangeTableViewTest.java
index 8d3a836..92b1c21 100644
--- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItSchemaChangeTableViewTest.java
+++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItSchemaChangeTableViewTest.java
@@ -30,6 +30,7 @@ import org.apache.ignite.schema.SchemaBuilders;
 import org.apache.ignite.schema.definition.ColumnDefinition;
 import org.apache.ignite.schema.definition.ColumnType;
 import org.apache.ignite.table.RecordView;
+import org.apache.ignite.table.Table;
 import org.apache.ignite.table.Tuple;
 import org.junit.jupiter.api.Test;
 
@@ -43,32 +44,32 @@ class ItSchemaChangeTableViewTest extends AbstractSchemaChangeTest {
     @Test
     public void testDropColumn() {
         List<Ignite> grid = startGrid();
-    
+        
         createTable(grid);
-    
+        
         RecordView<Tuple> tbl = grid.get(0).tables().table(TABLE).recordView();
-    
+        
         tbl.insert(Tuple.create().set("key", 1L).set("valInt", 111).set("valStr", "str"));
-    
+        
         dropColumn(grid, "valStr");
-    
+        
         // Check old row conversion.
         final Tuple keyTuple = Tuple.create().set("key", 1L);
-    
+        
         assertEquals(1, (Long) tbl.get(keyTuple).value("key"));
         assertEquals(111, (Integer) tbl.get(keyTuple).value("valInt"));
         assertThrows(IllegalArgumentException.class, () -> tbl.get(keyTuple).value("valStr"));
-    
+        
         // Check tuple of outdated schema.
         assertThrowsWithCause(SchemaMismatchException.class,
                 () -> tbl.insert(Tuple.create().set("key", 2L).set("valInt", -222).set("valStr", "str"))
         );
-    
+        
         // Check tuple of correct schema.
         tbl.insert(Tuple.create().set("key", 2L).set("valInt", 222));
-    
+        
         final Tuple keyTuple2 = Tuple.create().set("key", 2L);
-    
+        
         assertEquals(2, (Long) tbl.get(keyTuple2).value("key"));
         assertEquals(222, (Integer) tbl.get(keyTuple2).value("valInt"));
         assertThrows(IllegalArgumentException.class, () -> tbl.get(keyTuple2).value("valStr"));
@@ -82,11 +83,11 @@ class ItSchemaChangeTableViewTest extends AbstractSchemaChangeTest {
         List<Ignite> grid = startGrid();
         
         createTable(grid);
-    
+        
         RecordView<Tuple> tbl = grid.get(0).tables().table(TABLE).recordView();
-    
+        
         tbl.insert(Tuple.create().set("key", 1L).set("valInt", 111));
-    
+        
         assertThrowsWithCause(SchemaMismatchException.class,
                 () -> tbl.insert(Tuple.create().set("key", 1L).set("valInt", -111).set("valStrNew", "str"))
         );
@@ -118,34 +119,34 @@ class ItSchemaChangeTableViewTest extends AbstractSchemaChangeTest {
         List<Ignite> grid = startGrid();
         
         createTable(grid);
-    
+        
         RecordView<Tuple> tbl = grid.get(0).tables().table(TABLE).recordView();
-    
+        
         tbl.insert(Tuple.create().set("key", 1L).set("valInt", 111));
-    
+        
         assertThrowsWithCause(SchemaMismatchException.class,
                 () -> tbl.insert(Tuple.create().set("key", 2L).set("valRenamed", -222))
         );
-    
+        
         renameColumn(grid, "valInt", "valRenamed");
-    
+        
         // Check old row conversion.
         Tuple keyTuple1 = Tuple.create().set("key", 1L);
-    
+        
         assertEquals(1, (Long) tbl.get(keyTuple1).value("key"));
         assertEquals(111, (Integer) tbl.get(keyTuple1).value("valRenamed"));
         assertThrows(IllegalArgumentException.class, () -> tbl.get(keyTuple1).value("valInt"));
-    
+        
         // Check tuple of outdated schema.
         assertThrowsWithCause(SchemaMismatchException.class,
                 () -> tbl.insert(Tuple.create().set("key", 2L).set("valInt", -222))
         );
-    
+        
         // Check tuple of correct schema.
         tbl.insert(Tuple.create().set("key", 2L).set("valRenamed", 222));
-    
+        
         Tuple keyTuple2 = Tuple.create().set("key", 2L);
-    
+        
         assertEquals(2, (Long) tbl.get(keyTuple2).value("key"));
         assertEquals(222, (Integer) tbl.get(keyTuple2).value("valRenamed"));
         assertThrows(IllegalArgumentException.class, () -> tbl.get(keyTuple2).value("valInt"));
@@ -159,33 +160,33 @@ class ItSchemaChangeTableViewTest extends AbstractSchemaChangeTest {
         List<Ignite> grid = startGrid();
         
         createTable(grid);
-    
+        
         RecordView<Tuple> tbl = grid.get(0).tables().table(TABLE).recordView();
-    
+        
         tbl.insert(Tuple.create().set("key", 1L).set("valInt", 111));
-    
+        
         assertThrowsWithCause(SchemaMismatchException.class,
                 () -> tbl.insert(Tuple.create().set("key", 2L).set("val2", -222))
         );
-    
+        
         renameColumn(grid, "valInt", "val2");
         addColumn(grid, SchemaBuilders.column("valInt", ColumnType.INT32).asNullable().withDefaultValueExpression(-1).build());
-    
+        
         // Check old row conversion.
         Tuple keyTuple1 = Tuple.create().set("key", 1L);
-    
+        
         assertEquals(1, (Long) tbl.get(keyTuple1).value("key"));
         assertEquals(111, (Integer) tbl.get(keyTuple1).value("val2"));
         assertEquals(-1, (Integer) tbl.get(keyTuple1).value("valInt"));
-    
+        
         // Check tuple of outdated schema.
         assertNull(tbl.get(Tuple.create().set("key", 2L)));
-    
+        
         // Check tuple of correct schema.
         tbl.insert(Tuple.create().set("key", 2L).set("val2", 222));
-    
+        
         Tuple keyTuple2 = Tuple.create().set("key", 2L);
-    
+        
         assertEquals(2, (Long) tbl.get(keyTuple2).value("key"));
         assertEquals(222, (Integer) tbl.get(keyTuple2).value("val2"));
         assertEquals(-1, (Integer) tbl.get(keyTuple2).value("valInt"));
@@ -204,56 +205,56 @@ class ItSchemaChangeTableViewTest extends AbstractSchemaChangeTest {
                 .build();
         
         RecordView<Tuple> tbl = grid.get(0).tables().table(TABLE).recordView();
-    
+        
         tbl.insert(Tuple.create().set("key", 1L).set("valInt", 111));
-    
+        
         assertThrowsWithCause(SchemaMismatchException.class, () -> tbl.insert(
                 Tuple.create().set("key", 2L).set("val", "I'not exists"))
         );
         
         addColumn(grid, column);
-    
+        
         assertNull(tbl.get(Tuple.create().set("key", 2L)));
-    
+        
         tbl.insert(Tuple.create().set("key", 2L).set("valInt", 222).set("val", "string"));
-    
+        
         tbl.insert(Tuple.create().set("key", 3L).set("valInt", 333));
         
         dropColumn(grid, column.name());
-    
+        
         tbl.insert(Tuple.create().set("key", 4L).set("valInt", 444));
-    
+        
         assertThrowsWithCause(SchemaMismatchException.class, () -> tbl.insert(
                 Tuple.create().set("key", 4L).set("val", "I'm not exist"))
         );
-    
+        
         addColumn(grid, SchemaBuilders.column("val", ColumnType.string()).withDefaultValueExpression("default").build());
-    
+        
         tbl.insert(Tuple.create().set("key", 5L).set("valInt", 555));
-    
+        
         // Check old row conversion.
         Tuple keyTuple1 = Tuple.create().set("key", 1L);
-    
+        
         assertEquals(111, (Integer) tbl.get(keyTuple1).value("valInt"));
         assertEquals("default", tbl.get(keyTuple1).value("val"));
-    
+        
         Tuple keyTuple2 = Tuple.create().set("key", 2L);
-    
+        
         assertEquals(222, (Integer) tbl.get(keyTuple2).value("valInt"));
         assertEquals("default", tbl.get(keyTuple2).value("val"));
-    
+        
         Tuple keyTuple3 = Tuple.create().set("key", 3L);
-    
+        
         assertEquals(333, (Integer) tbl.get(keyTuple3).value("valInt"));
         assertEquals("default", tbl.get(keyTuple3).value("val"));
-    
+        
         Tuple keyTuple4 = Tuple.create().set("key", 4L);
-    
+        
         assertEquals(444, (Integer) tbl.get(keyTuple4).value("valInt"));
         assertEquals("default", tbl.get(keyTuple4).value("val"));
-    
+        
         Tuple keyTuple5 = Tuple.create().set("key", 5L);
-    
+        
         assertEquals(555, (Integer) tbl.get(keyTuple5).value("valInt"));
         assertEquals("default", tbl.get(keyTuple5).value("val"));
     }
@@ -270,36 +271,52 @@ class ItSchemaChangeTableViewTest extends AbstractSchemaChangeTest {
         RecordView<Tuple> tbl = grid.get(0).tables().table(TABLE).recordView();
         
         final String colName = "valStr";
-    
+        
         tbl.insert(Tuple.create().set("key", 1L).set("valInt", 111));
         
         changeDefault(grid, colName, (Supplier<Object> & Serializable) () -> "newDefault");
         addColumn(grid, SchemaBuilders.column("val", ColumnType.string()).withDefaultValueExpression("newDefault").build());
-    
+        
         tbl.insert(Tuple.create().set("key", 2L).set("valInt", 222));
-    
+        
         changeDefault(grid, colName, (Supplier<Object> & Serializable) () -> "brandNewDefault");
         changeDefault(grid, "val", (Supplier<Object> & Serializable) () -> "brandNewDefault");
-    
+        
         tbl.insert(Tuple.create().set("key", 3L).set("valInt", 333));
-    
+        
         // Check old row conversion.
         Tuple keyTuple1 = Tuple.create().set("key", 1L);
-    
+        
         assertEquals(111, (Integer) tbl.get(keyTuple1).value("valInt"));
         assertEquals("default", tbl.get(keyTuple1).value("valStr"));
         assertEquals("newDefault", tbl.get(keyTuple1).value("val"));
-    
+        
         Tuple keyTuple2 = Tuple.create().set("key", 2L);
-    
+        
         assertEquals(222, (Integer) tbl.get(keyTuple2).value("valInt"));
         assertEquals("newDefault", tbl.get(keyTuple2).value("valStr"));
         assertEquals("newDefault", tbl.get(keyTuple2).value("val"));
-    
+        
         Tuple keyTuple3 = Tuple.create().set("key", 3L);
-    
+        
         assertEquals(333, (Integer) tbl.get(keyTuple3).value("valInt"));
         assertEquals("brandNewDefault", tbl.get(keyTuple3).value("valStr"));
         assertEquals("brandNewDefault", tbl.get(keyTuple3).value("val"));
     }
+    
+    /**
+     * Check operation failed if unknown column found.
+     */
+    @Test
+    public void testStrictSchemaInsertRowOfNewSchema() {
+        List<Ignite> grid = startGrid();
+        
+        createTable(grid);
+        
+        Table tbl = grid.get(0).tables().table(TABLE);
+        
+        Tuple tuple = Tuple.create().set("key", 1L).set("unknownColumn", 10);
+        
+        assertThrowsWithCause(SchemaMismatchException.class, () -> tbl.recordView().insert(tuple));
+    }
 }
diff --git a/modules/schema/README.md b/modules/schema/README.md
index f9f6742..aba4ff3 100644
--- a/modules/schema/README.md
+++ b/modules/schema/README.md
@@ -15,7 +15,7 @@ for a long enough period of time to allow upgrade of all existing data stored in
               
 Given a schema evolution history, a row migration from version `N-k` to version `N` is a straightforward operation. 
 We identify fields that were dropped during the last k schema operations and fields that were added (taking into account
-default field values) and update the row based on the field modifications. Afterward, the updated row is written in 
+default field values) and update the row based on the field modifications. Afterward, the updated row is written in
 the schema version `N` layout format. The row upgrade may happen on read with an optional writeback or on next update. 
 Additionally, row upgrade in background is possible.
               
@@ -27,21 +27,6 @@ produce duplicates, and we assume PK has no duplicates.
 Additionally to adding and removing columns, it may be possible to allow for column type migrations when the type change 
 is non-ambiguous (a type upcast, e.g. Int8 → Int16, or by means of a certain expression, e,g, Int8 → String using 
 the `CAST` expression).
-
-### Dynamic schema expansion (live schema)
-Ignite can operate in two modes that provide different flexibility level and restrictions wrt object-to-schema mapping:
- * Strict mode. When a user attempts to insert/update an object to a table, Ignite checks that the object does not 
- contain any extra columns that are not present in the current table schema. If such columns are detected, Ignite will
- fail the operation requiring the user to manually update the schema before working with added columns.     
- * Live mode. When an object is inserted into a table, we attempt to 'fit' object fields to the schema columns. If the 
- object has some extra fields which are not present in the current schema, the schema is automatically updated to store 
- additional extra fields that are present in the object. If there are two concurrent live schema modifications, they can 
- either merge together if modifications are non-conflicting (e.g. adding disjoint sets of columns or adding columns with
- the same definition), or one of the modifications will fail (e.g. two columns with the same name, but conflicting type
- are being inserted). Live schema will try to automatically expand the schema even if there was an explicit drop column
- command executed right before the live schema expansion. **Live schema never drops columns during automatic schema 
- evolution.** If a schema has columns that were not fulfilled by object fields, they will be either kept `null` or 
- populated with defaults when provided, or the update will fail with an exception.
  
 ### Data Layout
 Data layout is documentation can be found [here](src/main/java/org/apache/ignite/internal/schema/README.md)
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/schema/marshaller/TupleMarshallerImpl.java b/modules/table/src/main/java/org/apache/ignite/internal/schema/marshaller/TupleMarshallerImpl.java
index 67842e7..072e375 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/schema/marshaller/TupleMarshallerImpl.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/schema/marshaller/TupleMarshallerImpl.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.schema.marshaller;
 
-import static org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter.convert;
 import static org.apache.ignite.internal.schema.marshaller.MarshallerUtil.getValueSize;
 
 import java.util.HashMap;
@@ -27,20 +26,12 @@ import java.util.Objects;
 import java.util.Set;
 import org.apache.ignite.internal.schema.Column;
 import org.apache.ignite.internal.schema.Columns;
-import org.apache.ignite.internal.schema.InvalidTypeException;
 import org.apache.ignite.internal.schema.SchemaAware;
 import org.apache.ignite.internal.schema.SchemaDescriptor;
 import org.apache.ignite.internal.schema.SchemaMismatchException;
 import org.apache.ignite.internal.schema.SchemaRegistry;
-import org.apache.ignite.internal.schema.configuration.SchemaConfigurationConverter;
 import org.apache.ignite.internal.schema.row.Row;
 import org.apache.ignite.internal.schema.row.RowAssembler;
-import org.apache.ignite.internal.table.InternalTable;
-import org.apache.ignite.internal.table.distributed.TableManager;
-import org.apache.ignite.schema.SchemaBuilders;
-import org.apache.ignite.schema.definition.ColumnDefinition;
-import org.apache.ignite.schema.definition.ColumnType;
-import org.apache.ignite.schema.definition.SchemaManagementMode;
 import org.apache.ignite.table.Tuple;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
@@ -55,23 +46,13 @@ public class TupleMarshallerImpl implements TupleMarshaller {
     /** Schema manager. */
     private final SchemaRegistry schemaReg;
     
-    /** Table manager. */
-    private final TableManager tblMgr;
-    
-    /** Internal table. */
-    private final InternalTable tbl;
-    
     /**
      * Creates tuple marshaller.
      *
-     * @param tblMgr    Table manager.
-     * @param tbl       Internal table.
      * @param schemaReg Schema manager.
      */
-    public TupleMarshallerImpl(TableManager tblMgr, InternalTable tbl, SchemaRegistry schemaReg) {
+    public TupleMarshallerImpl(SchemaRegistry schemaReg) {
         this.schemaReg = schemaReg;
-        this.tblMgr = tblMgr;
-        this.tbl = tbl;
     }
     
     /** {@inheritDoc} */
@@ -83,19 +64,10 @@ public class TupleMarshallerImpl implements TupleMarshaller {
             InternalTuple keyTuple0 = toInternalTuple(schema, tuple, true);
             InternalTuple valTuple0 = toInternalTuple(schema, tuple, false);
             
-            while (valTuple0.knownColumns() + keyTuple0.knownColumns() != tuple.columnCount()) {
-                if (tbl.schemaMode() == SchemaManagementMode.STRICT) {
-                    throw new SchemaMismatchException("Value doesn't match schema.");
-                }
-                
-                createColumns(extractColumnsType(tuple, extraColumnNames(tuple, schema)));
-                
-                assert schemaReg.lastSchemaVersion() > schema.version();
-                
-                schema = schemaReg.schema();
-                
-                keyTuple0 = toInternalTuple(schema, tuple, true);
-                valTuple0 = toInternalTuple(schema, tuple, false);
+            if (valTuple0.knownColumns() + keyTuple0.knownColumns() != tuple.columnCount()) {
+                throw new SchemaMismatchException(
+                        String.format("Tuple doesn't match schema: schemaVersion=%s, extraColumns=%s",
+                                schema.version(), extraColumnNames(tuple, schema)));
             }
             
             return buildRow(schema, keyTuple0, valTuple0);
@@ -113,27 +85,16 @@ public class TupleMarshallerImpl implements TupleMarshaller {
             InternalTuple keyTuple0 = toInternalTuple(schema, keyTuple, true);
             InternalTuple valTuple0 = toInternalTuple(schema, valTuple, false);
             
-            while (true) {
-                if (keyTuple0.knownColumns() < keyTuple.columnCount()) {
-                    throw new SchemaMismatchException("Key tuple contains extra columns: " + extraColumnNames(keyTuple, true, schema));
-                }
-    
-                if (valTuple == null || valTuple0.knownColumns() == valTuple.columnCount()) {
-                    break; // Nothing to do.
-                }
-    
-                if (tbl.schemaMode() == SchemaManagementMode.STRICT) {
-                    throw new SchemaMismatchException("Value doesn't match schema.");
-                }
-                
-                createColumns(extractColumnsType(valTuple, extraColumnNames(valTuple, false, schema)));
-                
-                assert schemaReg.lastSchemaVersion() > schema.version();
-                
-                schema = schemaReg.schema();
-                
-                keyTuple0 = toInternalTuple(schema, keyTuple, true);
-                valTuple0 = toInternalTuple(schema, valTuple, false);
+            if (keyTuple0.knownColumns() != keyTuple.columnCount()) {
+                throw new SchemaMismatchException(
+                        String.format("Key tuple doesn't match schema: schemaVersion=%s, extraColumns=%s",
+                                schema.version(), extraColumnNames(keyTuple, true, schema)));
+            }
+            
+            if (valTuple != null && valTuple0.knownColumns() != valTuple.columnCount()) {
+                throw new SchemaMismatchException(
+                        String.format("Value tuple doesn't match schema: schemaVersion=%s, extraColumns=%s",
+                                schema.version(), extraColumnNames(valTuple, false, schema)));
             }
             
             return buildRow(schema, keyTuple0, valTuple0);
@@ -183,7 +144,7 @@ public class TupleMarshallerImpl implements TupleMarshaller {
             final SchemaDescriptor schema = schemaReg.schema();
             
             InternalTuple keyTuple0 = toInternalTuple(schema, keyTuple, true);
-    
+            
             if (keyTuple0.knownColumns() < keyTuple.columnCount()) {
                 throw new SchemaMismatchException("Key tuple contains extra columns: " + extraColumnNames(keyTuple, true, schema));
             }
@@ -232,7 +193,7 @@ public class TupleMarshallerImpl implements TupleMarshaller {
                 Object val = tuple.valueOrDefault(col.name(), POISON_OBJECT);
                 
                 assert val != POISON_OBJECT;
-    
+                
                 if (val == null || columns.firstVarlengthColumn() < i) {
                     continue;
                 }
@@ -245,21 +206,21 @@ public class TupleMarshallerImpl implements TupleMarshaller {
                 final Column col = columns.column(i);
                 
                 Object val = tuple.valueOrDefault(col.name(), POISON_OBJECT);
-    
+                
                 if (val == POISON_OBJECT) {
                     if (keyFlag) {
                         throw new SchemaMismatchException("Missed key column: " + col.name());
                     }
-        
+                    
                     val = col.defaultValue();
-        
+                    
                     defaults.put(col.name(), val);
                 } else {
                     knownColumns++;
                 }
                 
                 col.validate(val);
-    
+                
                 if (val == null || columns.isFixedSize(i)) {
                     continue;
                 }
@@ -284,7 +245,7 @@ public class TupleMarshallerImpl implements TupleMarshaller {
         
         for (int i = 0, len = tuple.columnCount(); i < len; i++) {
             String colName = tuple.columnName(i);
-    
+            
             if (schema.column(colName) == null) {
                 cols.add(colName);
             }
@@ -309,7 +270,7 @@ public class TupleMarshallerImpl implements TupleMarshaller {
             String colName = tuple.columnName(i);
             
             Column col = schema.column(colName);
-    
+            
             if (col == null || schema.isKeyColumn(col.schemaIndex()) ^ keyTuple) {
                 cols.add(colName);
             }
@@ -319,38 +280,6 @@ public class TupleMarshallerImpl implements TupleMarshaller {
     }
     
     /**
-     * Extract column types from the tuple that are missed in schema.
-     *
-     * @param tuple    Tuple with column values.
-     * @param colNames Column names that type info to be extracted.
-     * @return Column types.
-     * @throws InvalidTypeException If failed to extract a type for tuple column value.
-     */
-    private Set<ColumnDefinition> extractColumnsType(Tuple tuple, Set<String> colNames) throws InvalidTypeException {
-        Set<ColumnDefinition> extraColumns = new HashSet<>();
-        
-        for (String colName : colNames) {
-            Object colValue = tuple.value(colName);
-    
-            if (colValue == null) {
-                // Can't detect type of 'null'
-                throw new InvalidTypeException("Live schema upgrade for 'null' value is not supported yet.");
-            }
-            
-            ColumnType colType = SchemaConfigurationConverter.columnType(colValue.getClass());
-    
-            if (colType == null) {
-                // No native support for type.
-                throw new InvalidTypeException("Live schema upgrade for type [" + colValue.getClass() + "] is not supported.");
-            }
-            
-            extraColumns.add(SchemaBuilders.column(colName, colType).asNullable().build());
-        }
-        
-        return extraColumns;
-    }
-    
-    /**
      * Creates {@link RowAssembler} for key-value tuples.
      *
      * @param schema   Schema.
@@ -380,23 +309,6 @@ public class TupleMarshallerImpl implements TupleMarshaller {
     }
     
     /**
-     * Updates the schema with new columns.
-     *
-     * @param newCols Columns to add.
-     */
-    private void createColumns(Set<ColumnDefinition> newCols) {
-        tblMgr.alterTable(tbl.tableName(), chng -> chng.changeColumns(cols -> {
-            int colIdx = chng.columns().size();
-            //TODO: IGNITE-15096 avoid 'colIdx' or replace with correct last colIdx.
-            
-            for (ColumnDefinition column : newCols) {
-                cols.create(String.valueOf(colIdx), colChg -> convert(column, colChg));
-                colIdx++;
-            }
-        }));
-    }
-    
-    /**
      * Internal tuple enriched original tuple with additional info.
      */
     private static class InternalTuple {
@@ -449,7 +361,7 @@ public class TupleMarshallerImpl implements TupleMarshaller {
          */
         Object value(String columnName) {
             Object val = tuple.valueOrDefault(columnName, POISON_OBJECT);
-    
+            
             if (val == POISON_OBJECT) {
                 return defaults.get(columnName);
             }
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/InternalTable.java b/modules/table/src/main/java/org/apache/ignite/internal/table/InternalTable.java
index f9800b2..d3f152c 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/InternalTable.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/InternalTable.java
@@ -24,7 +24,6 @@ import java.util.concurrent.Flow.Publisher;
 import org.apache.ignite.internal.schema.BinaryRow;
 import org.apache.ignite.internal.storage.engine.TableStorage;
 import org.apache.ignite.lang.IgniteUuid;
-import org.apache.ignite.schema.definition.SchemaManagementMode;
 import org.apache.ignite.tx.Transaction;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
@@ -56,18 +55,6 @@ public interface InternalTable extends AutoCloseable {
     @NotNull String tableName();
 
     /**
-     * Gets a schema mode of the table.
-     *
-     * @return Schema mode.
-     */
-    @NotNull SchemaManagementMode schemaMode();
-
-    /**
-     * Sets schema mode for the table.
-     */
-    void schema(SchemaManagementMode schemaMode);
-
-    /**
      * Asynchronously gets a row with same key columns values as given one from the table.
      *
      * @param keyRow Row with key columns set.
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/KeyValueBinaryViewImpl.java b/modules/table/src/main/java/org/apache/ignite/internal/table/KeyValueBinaryViewImpl.java
index bfdf230..af12a94 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/KeyValueBinaryViewImpl.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/KeyValueBinaryViewImpl.java
@@ -60,7 +60,7 @@ public class KeyValueBinaryViewImpl extends AbstractTableView implements KeyValu
         
         this.tblMgr = tblMgr;
         
-        marsh = new TupleMarshallerImpl(tblMgr, tbl, schemaReg);
+        marsh = new TupleMarshallerImpl(schemaReg);
     }
     
     /** {@inheritDoc} */
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/RecordBinaryViewImpl.java b/modules/table/src/main/java/org/apache/ignite/internal/table/RecordBinaryViewImpl.java
index 667f8ef..aadc84c 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/RecordBinaryViewImpl.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/RecordBinaryViewImpl.java
@@ -59,7 +59,7 @@ public class RecordBinaryViewImpl extends AbstractTableView implements RecordVie
     public RecordBinaryViewImpl(InternalTable tbl, SchemaRegistry schemaReg, TableManager tblMgr, @Nullable Transaction tx) {
         super(tbl, schemaReg, tx);
         
-        marsh = new TupleMarshallerImpl(tblMgr, tbl, schemaReg);
+        marsh = new TupleMarshallerImpl(schemaReg);
         
         this.tblMgr = tblMgr;
     }
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/TableImpl.java b/modules/table/src/main/java/org/apache/ignite/internal/table/TableImpl.java
index f34216d..65d7187 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/TableImpl.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/TableImpl.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.table;
 import org.apache.ignite.internal.schema.SchemaRegistry;
 import org.apache.ignite.internal.table.distributed.TableManager;
 import org.apache.ignite.lang.IgniteUuid;
-import org.apache.ignite.schema.definition.SchemaManagementMode;
 import org.apache.ignite.table.KeyValueView;
 import org.apache.ignite.table.RecordView;
 import org.apache.ignite.table.Table;
@@ -101,13 +100,4 @@ public class TableImpl implements Table {
     @Override public KeyValueView<Tuple, Tuple> keyValueView() {
         return new KeyValueBinaryViewImpl(tbl, schemaReg, tblMgr, null);
     }
-
-    /**
-     * Sets new schema management mode.
-     *
-     * @param schemaMode New schema management mode.
-     */
-    public void schemaMode(SchemaManagementMode schemaMode) {
-        this.tbl.schema(schemaMode);
-    }
 }
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/storage/InternalTableImpl.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/storage/InternalTableImpl.java
index fef6c00..f00da16 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/storage/InternalTableImpl.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/storage/InternalTableImpl.java
@@ -63,7 +63,6 @@ import org.apache.ignite.lang.LoggerMessageHelper;
 import org.apache.ignite.network.NetworkAddress;
 import org.apache.ignite.raft.client.Peer;
 import org.apache.ignite.raft.client.service.RaftGroupService;
-import org.apache.ignite.schema.definition.SchemaManagementMode;
 import org.apache.ignite.tx.Transaction;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
@@ -94,9 +93,6 @@ public class InternalTableImpl implements InternalTable {
     /** Resolver that resolves a network address to node id. */
     private final Function<NetworkAddress, String> netAddrResolver;
 
-    /** Table schema mode. */
-    private volatile SchemaManagementMode schemaMode;
-
     /** Storage for table data. */
     private final TableStorage tableStorage;
 
@@ -122,8 +118,6 @@ public class InternalTableImpl implements InternalTable {
         this.partitions = partitions;
         this.netAddrResolver = netAddrResolver;
         this.tableStorage = tableStorage;
-
-        this.schemaMode = SchemaManagementMode.STRICT;
     }
 
     /** {@inheritDoc} */
@@ -152,18 +146,6 @@ public class InternalTableImpl implements InternalTable {
 
     /** {@inheritDoc} */
     @Override
-    public SchemaManagementMode schemaMode() {
-        return schemaMode;
-    }
-
-    /** {@inheritDoc} */
-    @Override
-    public void schema(SchemaManagementMode schemaMode) {
-        this.schemaMode = schemaMode;
-    }
-
-    /** {@inheritDoc} */
-    @Override
     public CompletableFuture<BinaryRow> get(BinaryRow keyRow, Transaction tx) {
         return partitionMap.get(partId(keyRow)).<SingleRowResponse>run(new GetCommand(keyRow))
                 .thenApply(SingleRowResponse::getValue);
diff --git a/modules/table/src/test/java/org/apache/ignite/internal/benchmarks/TupleMarshallerFixlenOnlyBenchmark.java b/modules/table/src/test/java/org/apache/ignite/internal/benchmarks/TupleMarshallerFixlenOnlyBenchmark.java
index b9dd1e7..5be02bc 100644
--- a/modules/table/src/test/java/org/apache/ignite/internal/benchmarks/TupleMarshallerFixlenOnlyBenchmark.java
+++ b/modules/table/src/test/java/org/apache/ignite/internal/benchmarks/TupleMarshallerFixlenOnlyBenchmark.java
@@ -105,7 +105,7 @@ public class TupleMarshallerFixlenOnlyBenchmark {
                         .toArray(Column[]::new)
         );
 
-        marshaller = new TupleMarshallerImpl(null, null, new SchemaRegistryImpl(v -> null) {
+        marshaller = new TupleMarshallerImpl(new SchemaRegistryImpl(v -> null) {
             @Override
             public SchemaDescriptor schema() {
                 return schema;
diff --git a/modules/table/src/test/java/org/apache/ignite/internal/benchmarks/TupleMarshallerVarlenOnlyBenchmark.java b/modules/table/src/test/java/org/apache/ignite/internal/benchmarks/TupleMarshallerVarlenOnlyBenchmark.java
index ec00aaf..8d3ea6c 100644
--- a/modules/table/src/test/java/org/apache/ignite/internal/benchmarks/TupleMarshallerVarlenOnlyBenchmark.java
+++ b/modules/table/src/test/java/org/apache/ignite/internal/benchmarks/TupleMarshallerVarlenOnlyBenchmark.java
@@ -119,7 +119,7 @@ public class TupleMarshallerVarlenOnlyBenchmark {
                         .toArray(Column[]::new)
         );
 
-        marshaller = new TupleMarshallerImpl(null, null, new SchemaRegistryImpl(v -> null) {
+        marshaller = new TupleMarshallerImpl(new SchemaRegistryImpl(v -> null) {
             @Override
             public SchemaDescriptor schema() {
                 return schema;
diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/MutableRowTupleAdapterTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/MutableRowTupleAdapterTest.java
index a963188..4ba3dc4 100644
--- a/modules/table/src/test/java/org/apache/ignite/internal/table/MutableRowTupleAdapterTest.java
+++ b/modules/table/src/test/java/org/apache/ignite/internal/table/MutableRowTupleAdapterTest.java
@@ -61,7 +61,6 @@ import org.apache.ignite.internal.schema.marshaller.TupleMarshallerImpl;
 import org.apache.ignite.internal.schema.row.Row;
 import org.apache.ignite.internal.table.impl.DummySchemaManagerImpl;
 import org.apache.ignite.internal.testframework.IgniteTestUtils;
-import org.apache.ignite.schema.definition.SchemaManagementMode;
 import org.apache.ignite.table.Tuple;
 import org.junit.jupiter.api.Test;
 import org.mockito.Mockito;
@@ -72,10 +71,6 @@ import org.mockito.Mockito;
  * <p>Should be in sync with org.apache.ignite.client.ClientTupleBuilderTest.
  */
 public class MutableRowTupleAdapterTest {
-    /** Mocked table. */
-    private InternalTable tbl = Mockito.when(Mockito.mock(InternalTable.class).schemaMode()).thenReturn(SchemaManagementMode.STRICT)
-            .getMock();
-    
     /** Schema descriptor. */
     private SchemaDescriptor schema = new SchemaDescriptor(
             42,
@@ -217,7 +212,7 @@ public class MutableRowTupleAdapterTest {
                 .set("name", "Shirt")
                 .set("price", 5.99d);
         
-        TupleMarshaller marshaller = new TupleMarshallerImpl(null, tbl, new DummySchemaManagerImpl(schema));
+        TupleMarshaller marshaller = new TupleMarshallerImpl(new DummySchemaManagerImpl(schema));
         
         Row row = new Row(schema, new ByteBufferRow(marshaller.marshal(original).bytes()));
         
@@ -243,7 +238,7 @@ public class MutableRowTupleAdapterTest {
     
     @Test
     public void testRowTupleMutability() throws TupleMarshallerException {
-        TupleMarshaller marshaller = new TupleMarshallerImpl(null, tbl, new DummySchemaManagerImpl(schema));
+        TupleMarshaller marshaller = new TupleMarshallerImpl(new DummySchemaManagerImpl(schema));
         
         Row row = new Row(schema, new ByteBufferRow(marshaller.marshal(Tuple.create().set("id", 1L).set("name", "Shirt")).bytes()));
         
@@ -270,7 +265,7 @@ public class MutableRowTupleAdapterTest {
     
     @Test
     public void testKeyValueTupleMutability() throws TupleMarshallerException {
-        TupleMarshaller marshaller = new TupleMarshallerImpl(null, tbl, new DummySchemaManagerImpl(schema));
+        TupleMarshaller marshaller = new TupleMarshallerImpl(new DummySchemaManagerImpl(schema));
         
         Row row = new Row(schema, new ByteBufferRow(marshaller.marshal(Tuple.create().set("id", 1L).set("name", "Shirt")).bytes()));
         
@@ -299,7 +294,7 @@ public class MutableRowTupleAdapterTest {
     
     @Test
     public void testRowTupleSchemaAwareness() throws TupleMarshallerException {
-        TupleMarshaller marshaller = new TupleMarshallerImpl(null, tbl, new DummySchemaManagerImpl(schema));
+        TupleMarshaller marshaller = new TupleMarshallerImpl(new DummySchemaManagerImpl(schema));
         
         Row row = new Row(schema, new ByteBufferRow(marshaller.marshal(Tuple.create().set("id", 1L).set("name", "Shirt")).bytes()));
         
@@ -322,7 +317,7 @@ public class MutableRowTupleAdapterTest {
     
     @Test
     public void testKeyValueTupleSchemaAwareness() throws TupleMarshallerException {
-        TupleMarshaller marshaller = new TupleMarshallerImpl(null, tbl, new DummySchemaManagerImpl(schema));
+        TupleMarshaller marshaller = new TupleMarshallerImpl(new DummySchemaManagerImpl(schema));
         
         Row row = new Row(schema, new ByteBufferRow(marshaller.marshal(Tuple.create().set("id", 1L).set("name", "Shirt")).bytes()));
         
@@ -349,7 +344,7 @@ public class MutableRowTupleAdapterTest {
     public void testVariousColumnTypes() throws TupleMarshallerException {
         Random rnd = new Random();
         
-        TupleMarshaller marshaller = new TupleMarshallerImpl(null, tbl, new DummySchemaManagerImpl(fullSchema));
+        TupleMarshaller marshaller = new TupleMarshallerImpl(new DummySchemaManagerImpl(fullSchema));
         
         Tuple tuple = Tuple.create()
                 .set("valByteCol", (byte) 1)
@@ -401,7 +396,7 @@ public class MutableRowTupleAdapterTest {
                 .set("valNumberCol", BigInteger.valueOf(rnd.nextLong()))
                 .set("valDecimalCol", BigDecimal.valueOf(rnd.nextLong(), 5));
         
-        TupleMarshaller marshaller = new TupleMarshallerImpl(null, tbl, new DummySchemaManagerImpl(fullSchema));
+        TupleMarshaller marshaller = new TupleMarshallerImpl(new DummySchemaManagerImpl(fullSchema));
         
         Row row = new Row(fullSchema, new ByteBufferRow(marshaller.marshal(tup1).bytes()));
         
@@ -436,7 +431,7 @@ public class MutableRowTupleAdapterTest {
         Tuple tuple = Tuple.create(valTuple).set(keyTuple.columnName(0), keyTuple.value(0));
         
         // Check tuples backed with Row.
-        TupleMarshaller marshaller = new TupleMarshallerImpl(null, tbl, new DummySchemaManagerImpl(fullSchema));
+        TupleMarshaller marshaller = new TupleMarshallerImpl(new DummySchemaManagerImpl(fullSchema));
         
         Row row = new Row(fullSchema, new ByteBufferRow(marshaller.marshal(keyTuple, valTuple).bytes()));
         
@@ -519,7 +514,7 @@ public class MutableRowTupleAdapterTest {
                 .set("valNumberCol", BigInteger.valueOf(rnd.nextLong()))
                 .set("valDecimalCol", BigDecimal.valueOf(rnd.nextLong(), 5));
         
-        TupleMarshaller marshaller = new TupleMarshallerImpl(null, tbl, new DummySchemaManagerImpl(fullSchema));
+        TupleMarshaller marshaller = new TupleMarshallerImpl(new DummySchemaManagerImpl(fullSchema));
         
         Row row = new Row(fullSchema, new ByteBufferRow(marshaller.marshal(key1, val1).bytes()));
         
@@ -566,7 +561,7 @@ public class MutableRowTupleAdapterTest {
                     .set("id", 3L)
                     .set("name", "Shirt");
             
-            TupleMarshaller marshaller = new TupleMarshallerImpl(null, tbl, new DummySchemaManagerImpl(schema));
+            TupleMarshaller marshaller = new TupleMarshallerImpl(new DummySchemaManagerImpl(schema));
             
             return TableRow.tuple(new Row(schema, new ByteBufferRow(marshaller.marshal(original).bytes())));
         } catch (TupleMarshallerException e) {
diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/StrictSchemaOperationsTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/SchemaValidationTest.java
similarity index 98%
rename from modules/table/src/test/java/org/apache/ignite/internal/table/StrictSchemaOperationsTest.java
rename to modules/table/src/test/java/org/apache/ignite/internal/table/SchemaValidationTest.java
index 7eb063d..0998d73 100644
--- a/modules/table/src/test/java/org/apache/ignite/internal/table/StrictSchemaOperationsTest.java
+++ b/modules/table/src/test/java/org/apache/ignite/internal/table/SchemaValidationTest.java
@@ -35,9 +35,9 @@ import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.function.Executable;
 
 /**
- * Check data by strict schema.
+ * Checks if data compliant with the schema, otherwise the correct exception is thrown.
  */
-public class StrictSchemaOperationsTest {
+public class SchemaValidationTest {
     @Test
     public void columnNotExist() {
         SchemaDescriptor schema = new SchemaDescriptor(
@@ -50,7 +50,7 @@ public class StrictSchemaOperationsTest {
         
         assertThrowsWithCause(SchemaMismatchException.class, () -> recView.insert(Tuple.create().set("id", 0L).set("invalidCol", 0)));
     }
-
+    
     @Test
     public void schemaMismatch() {
         SchemaDescriptor schema = new SchemaDescriptor(
@@ -78,7 +78,7 @@ public class StrictSchemaOperationsTest {
         assertThrowsWithCause(SchemaMismatchException.class, () -> tbl.keyValueView().put(Tuple.create().set("id", 0L).set("affId", 1L),
                 Tuple.create().set("id", 0L).set("val", 0L)));
     }
-
+    
     @Test
     public void typeMismatch() {
         SchemaDescriptor schema = new SchemaDescriptor(
@@ -102,7 +102,7 @@ public class StrictSchemaOperationsTest {
         assertThrowsWithCause(InvalidTypeException.class,
                 () -> tbl.insert(Tuple.create().set("id", 0L).set("valBytes", new byte[]{0, 1, 2, 3})));
     }
-
+    
     @Test
     public void stringTypeMatch() {
         SchemaDescriptor schema = new SchemaDescriptor(
@@ -126,7 +126,7 @@ public class StrictSchemaOperationsTest {
         // Check string 3 char length and 9 bytes.
         tbl.insert(tuple.set("valString", "我是谁"));
     }
-
+    
     @Test
     public void bytesTypeMatch() {
         SchemaDescriptor schema = new SchemaDescriptor(
diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java b/modules/table/src/test/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java
index 37c6c3a..5de2605 100644
--- a/modules/table/src/test/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java
+++ b/modules/table/src/test/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java
@@ -33,7 +33,6 @@ import org.apache.ignite.internal.storage.engine.TableStorage;
 import org.apache.ignite.internal.table.InternalTable;
 import org.apache.ignite.lang.IgniteInternalException;
 import org.apache.ignite.lang.IgniteUuid;
-import org.apache.ignite.schema.definition.SchemaManagementMode;
 import org.apache.ignite.tx.Transaction;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
@@ -121,17 +120,6 @@ public class DummyInternalTableImpl implements InternalTable {
 
     /** {@inheritDoc} */
     @Override
-    public @NotNull SchemaManagementMode schemaMode() {
-        return SchemaManagementMode.STRICT;
-    }
-
-    /** {@inheritDoc} */
-    @Override
-    public void schema(SchemaManagementMode schemaMode) {
-    }
-
-    /** {@inheritDoc} */
-    @Override
     public CompletableFuture<BinaryRow> get(@NotNull BinaryRow row, Transaction tx) {
         assert row != null;
 
diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/type/NumericTypesSerializerTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/type/NumericTypesSerializerTest.java
index 70d56f9..cb0e988 100644
--- a/modules/table/src/test/java/org/apache/ignite/internal/table/type/NumericTypesSerializerTest.java
+++ b/modules/table/src/test/java/org/apache/ignite/internal/table/type/NumericTypesSerializerTest.java
@@ -35,17 +35,14 @@ import org.apache.ignite.internal.schema.marshaller.TupleMarshaller;
 import org.apache.ignite.internal.schema.marshaller.TupleMarshallerException;
 import org.apache.ignite.internal.schema.marshaller.TupleMarshallerImpl;
 import org.apache.ignite.internal.schema.row.Row;
-import org.apache.ignite.internal.table.InternalTable;
 import org.apache.ignite.internal.table.impl.DummySchemaManagerImpl;
 import org.apache.ignite.internal.util.Pair;
-import org.apache.ignite.schema.definition.SchemaManagementMode;
 import org.apache.ignite.table.Tuple;
 import org.jetbrains.annotations.NotNull;
 import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.params.ParameterizedTest;
 import org.junit.jupiter.params.provider.MethodSource;
-import org.mockito.Mockito;
 
 /**
  * Check numeric typed columns serialization.
@@ -57,10 +54,6 @@ public class NumericTypesSerializerTest {
     /** Schema descriptor. */
     private SchemaDescriptor schema;
     
-    /** Mocked table. */
-    private InternalTable tbl = Mockito.when(Mockito.mock(InternalTable.class).schemaMode()).thenReturn(SchemaManagementMode.STRICT)
-            .getMock();
-    
     /**
      * Returns list of BigInteger pairs for test.
      */
@@ -128,7 +121,7 @@ public class NumericTypesSerializerTest {
                 }
         );
         
-        TupleMarshaller marshaller = new TupleMarshallerImpl(null, tbl, new DummySchemaManagerImpl(schema));
+        TupleMarshaller marshaller = new TupleMarshallerImpl(new DummySchemaManagerImpl(schema));
         
         final Tuple tup = createTuple().set("key", rnd.nextLong()).set("number1", pair.getFirst()).set("number2", pair.getSecond());
         
@@ -145,7 +138,7 @@ public class NumericTypesSerializerTest {
                 new Column[]{new Column("number1", NativeTypes.numberOf(5), false)}
         );
         
-        TupleMarshaller marshaller = new TupleMarshallerImpl(null, tbl, new DummySchemaManagerImpl(schema));
+        TupleMarshaller marshaller = new TupleMarshallerImpl(new DummySchemaManagerImpl(schema));
         
         final Tuple badTup = createTuple().set("key", rnd.nextLong());
         
@@ -171,7 +164,7 @@ public class NumericTypesSerializerTest {
         
         final Tuple badTup = createTuple().set("key", rnd.nextLong());
         
-        TupleMarshaller marshaller = new TupleMarshallerImpl(null, tbl, new DummySchemaManagerImpl(schema));
+        TupleMarshaller marshaller = new TupleMarshallerImpl(new DummySchemaManagerImpl(schema));
         
         assertThrows(TupleMarshallerException.class,
                 () -> marshaller.marshal(badTup.set("decimalCol", new BigDecimal("123456789.0123"))),
@@ -204,7 +197,7 @@ public class NumericTypesSerializerTest {
         //representation of "0000" value.
         final Tuple tup = createTuple().set("key", rnd.nextLong()).set("decimalCol", new BigDecimal("0E+3"));
         
-        TupleMarshaller marshaller = new TupleMarshallerImpl(null, tbl, new DummySchemaManagerImpl(schema));
+        TupleMarshaller marshaller = new TupleMarshallerImpl(new DummySchemaManagerImpl(schema));
         
         final Row row = marshaller.marshal(tup);
         
@@ -229,7 +222,7 @@ public class NumericTypesSerializerTest {
                 .set("key", rnd.nextLong())
                 .set("decimalCol1", new BigDecimal(decimalStr));
         
-        TupleMarshaller marshaller = new TupleMarshallerImpl(null, tbl, new DummySchemaManagerImpl(schema));
+        TupleMarshaller marshaller = new TupleMarshallerImpl(new DummySchemaManagerImpl(schema));
         
         final Row row = marshaller.marshal(tup);
         
@@ -250,7 +243,7 @@ public class NumericTypesSerializerTest {
                 .set("key", rnd.nextLong())
                 .set("decimalCol", BigDecimal.valueOf(123, Integer.MAX_VALUE));
         
-        TupleMarshaller marshaller = new TupleMarshallerImpl(null, tbl, new DummySchemaManagerImpl(schema));
+        TupleMarshaller marshaller = new TupleMarshallerImpl(new DummySchemaManagerImpl(schema));
         
         final Row row = marshaller.marshal(tup);
         
@@ -271,7 +264,7 @@ public class NumericTypesSerializerTest {
                 }
         );
         
-        TupleMarshaller marshaller = new TupleMarshallerImpl(null, tbl, new DummySchemaManagerImpl(schema));
+        TupleMarshaller marshaller = new TupleMarshallerImpl(new DummySchemaManagerImpl(schema));
         
         long randomKey = rnd.nextLong();