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/04/12 13:47:44 UTC

[ignite-3] 01/01: Initial schema manager implementation. Minor refactoring.

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

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

commit 50e450797da15aa2232a0991b1dd68d6a31412e5
Author: Andrew Mashenkov <an...@gmail.com>
AuthorDate: Mon Apr 12 16:47:14 2021 +0300

    Initial schema manager implementation.
    Minor refactoring.
---
 .../SchemaRegistrationConflictException.java}      |  36 +----
 .../internal/table/schema/SchemaRegistry.java      | 174 +++++++++++++++++++++
 .../table/schema/SchemaRegistryException.java}     |  40 ++---
 .../table/schema/TableSchemaManagerImpl.java       | 123 +++++++++++++++
 .../table/impl/DummyInternalTableImpl.java         |   2 +-
 .../table/impl/DummySchemaManagerImpl.java         |   2 +-
 .../internal/table/schema/SchemaManagerTest.java   |  98 ++++++++++++
 .../test/java/org/apache/ignite/table/Example.java |   2 +-
 .../apache/ignite/table/KVViewOperationsTest.java  |   4 +-
 .../table/TableBinaryViewOperationsTest.java       |   4 +-
 10 files changed, 424 insertions(+), 61 deletions(-)

diff --git a/modules/table/src/test/java/org/apache/ignite/table/impl/DummySchemaManagerImpl.java b/modules/table/src/main/java/org/apache/ignite/internal/table/schema/SchemaRegistrationConflictException.java
similarity index 51%
copy from modules/table/src/test/java/org/apache/ignite/table/impl/DummySchemaManagerImpl.java
copy to modules/table/src/main/java/org/apache/ignite/internal/table/schema/SchemaRegistrationConflictException.java
index 01df2fe..adcc185 100644
--- a/modules/table/src/test/java/org/apache/ignite/table/impl/DummySchemaManagerImpl.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/schema/SchemaRegistrationConflictException.java
@@ -15,41 +15,21 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.table.impl;
+package org.apache.ignite.internal.table.schema;
 
-import org.apache.ignite.internal.schema.SchemaDescriptor;
-import org.apache.ignite.internal.table.TableSchemaManager;
-import org.jetbrains.annotations.NotNull;
+import org.apache.ignite.lang.IgniteInternalException;
 
 /**
- * Dummy schema manager for tests.
+ * Schema registration conflict exception is thown if
+ * registering schema's number was alredy registered earlier.
  */
-public class DummySchemaManagerImpl implements TableSchemaManager {
-    /** Schema. */
-    private final SchemaDescriptor schema;
-
+public class SchemaRegistrationConflictException extends IgniteInternalException {
     /**
      * Constructor.
      *
-     * @param schema Schema descriptor.
+     * @param msg Message.
      */
-    public DummySchemaManagerImpl(@NotNull SchemaDescriptor schema) {
-        assert schema != null;
-
-        this.schema = schema;
-    }
-
-    /** {@inheritDoc} */
-    @Override public SchemaDescriptor schema() {
-        return schema;
-    }
-
-    /** {@inheritDoc} */
-    @Override public SchemaDescriptor schema(int ver) {
-        assert ver >= 0;
-
-        assert schema.version() == ver;
-
-        return schema;
+    public SchemaRegistrationConflictException(String msg) {
+        super(msg);
     }
 }
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/schema/SchemaRegistry.java b/modules/table/src/main/java/org/apache/ignite/internal/table/schema/SchemaRegistry.java
new file mode 100644
index 0000000..015ea5e
--- /dev/null
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/schema/SchemaRegistry.java
@@ -0,0 +1,174 @@
+/*
+ * 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.table.schema;
+
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentSkipListMap;
+import org.apache.ignite.internal.schema.SchemaDescriptor;
+
+/**
+ * Holds schema descriptors for actual schema versions.
+ * <p>
+ * Schemas MUST be registered in a version ascending order incrementing by {@code 1} with NO gaps,
+ * otherwise an exception will be thrown. The version numbering starts from the {@code 1}.
+ * <p>
+ * After some table maintenance process some first versions may become outdated and can be safely cleaned up
+ * if the process guarantees the table no longer has a data of these versions.
+ *
+ * @implSpec The changes in between two arbitrary actual versions MUST NOT be lost.
+ * Thus, schema versions can only be removed from the beginning.
+ * @implSpec Initial schema history MAY be registered without the first outdate versions
+ * that could be cleaned up earlier.
+ */
+public class SchemaRegistry {
+    /** Initial schema version. */
+    public static final int INITIAL_SCHEMA_VERSION = -1;
+
+    /** Latest actual schemas. */
+    private final ConcurrentSkipListMap<Integer, SchemaDescriptor> history = new ConcurrentSkipListMap<>();
+
+    /** Last registered version. */
+    private volatile int lastVer;
+
+    /**
+     * Default constructor.
+     */
+    public SchemaRegistry() {
+        lastVer = INITIAL_SCHEMA_VERSION;
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param history Schema history.
+     */
+    public SchemaRegistry(List<SchemaDescriptor> history) {
+        if (history.isEmpty())
+            lastVer = INITIAL_SCHEMA_VERSION;
+        else {
+            validateSchemaHistory(history);
+
+            history.forEach(d -> this.history.put(d.version(), d));
+
+            lastVer = history.get(history.size() - 1).version();
+        }
+    }
+
+    /**
+     * @param history Schema history.
+     * @throws SchemaRegistryException If history is invalid.
+     */
+    private void validateSchemaHistory(List<SchemaDescriptor> history) {
+        if (history.isEmpty())
+            return;
+
+        int prevVer = Objects.requireNonNull(history.get(0), "Schema descriptor can't be null.").version();
+
+        assert prevVer > 0;
+
+        for (int i = 1; i < history.size(); i++) {
+            final SchemaDescriptor desc = Objects.requireNonNull(history.get(1), "Schema descriptor can't be null.");
+
+            if (desc.version() == (++prevVer))
+                throw new SchemaRegistryException("Illegal schema version: expected=" + prevVer + ", actual=" + desc.version());
+        }
+    }
+
+    /**
+     * Gets schema descriptor for given version.
+     *
+     * @param ver Schema version to get descriptor for.
+     * @return Schema descriptor.
+     */
+    public SchemaDescriptor schema(int ver) {
+        final SchemaDescriptor desc = history.get(ver);
+
+        if (desc != null)
+            return desc;
+
+        if (lastVer < ver || ver <= 0)
+            throw new SchemaRegistryException("Incorrect schema version requested: " + ver);
+
+        assert history.isEmpty() || ver < history.firstKey();
+
+        throw new SchemaRegistryException("Outdated schema version requested: " + ver);
+    }
+
+    /**
+     * Gets schema descriptor for the latest version.
+     *
+     * @return Schema descriptor.
+     */
+    public SchemaDescriptor schema() {
+        final int lastVer0 = lastVer;
+
+        final SchemaDescriptor desc = history.get(lastVer0);
+
+        if (desc != null)
+            return desc;
+
+        if (lastVer0 == INITIAL_SCHEMA_VERSION)
+            throw new SchemaRegistryException("Table schema was not initialized yet.");
+
+        assert lastVer0 < history.firstKey();
+
+        throw new SchemaRegistryException("Outdated schema version requested: " + lastVer0);
+    }
+
+    /**
+     * @return Last known schema version.
+     */
+    public int lastSchemaVersion() {
+        return lastVer;
+    }
+
+    /**
+     * Register new schema.
+     *
+     * @param desc Schema descriptor.
+     * @throws SchemaRegistrationConflictException If schema of provided version was already registered.
+     * @throws SchemaRegistryException If schema of incorrect version provided.
+     */
+    public void registerSchema(SchemaDescriptor desc) {
+        if (lastVer >= desc.version())
+            throw new SchemaRegistrationConflictException("Schema with given version is already exists: " + desc.version());
+        else if (desc.version() != lastVer + 1 && lastVer != INITIAL_SCHEMA_VERSION)
+            throw new SchemaRegistryException("Illegal schema version: expected=" + (lastVer + 1) + ", actual=" + desc.version());
+
+        history.put(desc.version(), desc);
+
+        lastVer = desc.version();
+    }
+
+    /**
+     * Cleanup history prior to given schema version.
+     *
+     * @param ver First actual schema version.
+     * @throws SchemaRegistryException If incorrect schema version provided.
+     */
+    public void removeUntil(int ver) {
+        assert INITIAL_SCHEMA_VERSION != lastVer;
+
+        if (lastVer < ver || ver <= 0)
+            throw new SchemaRegistryException("Incorrect schema version provided: " + ver);
+
+        history.keySet().stream().takeWhile(k -> k < ver).forEach(history::remove);
+    }
+
+}
diff --git a/modules/table/src/test/java/org/apache/ignite/table/impl/DummySchemaManagerImpl.java b/modules/table/src/main/java/org/apache/ignite/internal/table/schema/SchemaRegistryException.java
similarity index 51%
copy from modules/table/src/test/java/org/apache/ignite/table/impl/DummySchemaManagerImpl.java
copy to modules/table/src/main/java/org/apache/ignite/internal/table/schema/SchemaRegistryException.java
index 01df2fe..5b41260 100644
--- a/modules/table/src/test/java/org/apache/ignite/table/impl/DummySchemaManagerImpl.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/schema/SchemaRegistryException.java
@@ -15,41 +15,29 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.table.impl;
+package org.apache.ignite.internal.table.schema;
 
-import org.apache.ignite.internal.schema.SchemaDescriptor;
-import org.apache.ignite.internal.table.TableSchemaManager;
-import org.jetbrains.annotations.NotNull;
+import org.apache.ignite.lang.IgniteInternalException;
 
 /**
- * Dummy schema manager for tests.
+ * Schema registration exception.
  */
-public class DummySchemaManagerImpl implements TableSchemaManager {
-    /** Schema. */
-    private final SchemaDescriptor schema;
-
+public class SchemaRegistryException extends IgniteInternalException {
     /**
      * Constructor.
      *
-     * @param schema Schema descriptor.
+     * @param msg Message.
      */
-    public DummySchemaManagerImpl(@NotNull SchemaDescriptor schema) {
-        assert schema != null;
-
-        this.schema = schema;
+    public SchemaRegistryException(String msg) {
+        super(msg);
     }
 
-    /** {@inheritDoc} */
-    @Override public SchemaDescriptor schema() {
-        return schema;
-    }
-
-    /** {@inheritDoc} */
-    @Override public SchemaDescriptor schema(int ver) {
-        assert ver >= 0;
-
-        assert schema.version() == ver;
-
-        return schema;
+    /**
+     * Constructor.
+     *
+     * @param cause Cause.
+     */
+    public SchemaRegistryException(Throwable cause) {
+        super(cause);
     }
 }
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/schema/TableSchemaManagerImpl.java b/modules/table/src/main/java/org/apache/ignite/internal/table/schema/TableSchemaManagerImpl.java
new file mode 100644
index 0000000..1ca3ed1
--- /dev/null
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/schema/TableSchemaManagerImpl.java
@@ -0,0 +1,123 @@
+/*
+ * 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.table.schema;
+
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import org.apache.ignite.internal.schema.Column;
+import org.apache.ignite.internal.schema.SchemaDescriptor;
+import org.apache.ignite.internal.table.TableSchemaManager;
+
+import static org.apache.ignite.internal.table.schema.SchemaRegistry.INITIAL_SCHEMA_VERSION;
+
+/**
+ * Table schema manager component.
+ */
+public class TableSchemaManagerImpl implements TableSchemaManager {
+    /** Stub. */
+    private static final Column[] EMPTY_COLS_ARR = new Column[0];
+
+    /** Local registry for schema. */
+    private final SchemaRegistry schemaReg;
+
+    private final Map<UUID, Future<SchemaDescriptor>> pendingRegistrations = new ConcurrentHashMap<>();
+
+    /**
+     * Constructor.
+     */
+    public TableSchemaManagerImpl() {
+        this.schemaReg = new SchemaRegistry();
+    }
+
+    /**
+     * Gets schema description for version.
+     *
+     * @param ver Schema version to get descriptor for.
+     * @return Schema descriptor.
+     */
+    @Override public SchemaDescriptor schema(int ver) {
+        return schemaReg.schema(ver);
+    }
+
+    /**
+     * Gets schema description for version.
+     *
+     * @return Schema descriptor.
+     */
+    @Override public SchemaDescriptor schema() {
+        return schemaReg.schema();
+    }
+
+    /**
+     * Registers schema in local registry.
+     *
+     * @param desc Schema descriptor.
+     */
+    private void registerSchemaLocal(SchemaDescriptor desc) {
+        schemaReg.registerSchema(desc);
+    }
+
+    /**
+     * Metastore callback is triggered when new schema was registered in grid.
+     *
+     * @param schemaDesc Schema descriptor.
+     */
+    public void onSchemaUpdated(SchemaDescriptor schemaDesc) {
+        registerSchemaLocal(schemaDesc);
+    }
+
+    /**
+     * Compares schemas.
+     *
+     * @param expected Expected schema.
+     * @param actual Actual schema.
+     * @return {@code True} if schemas are equal, {@code false} otherwise.
+     */
+    static boolean equalSchemas(SchemaDescriptor expected, SchemaDescriptor actual) {
+        if (expected.keyColumns().length() != actual.keyColumns().length() ||
+            expected.valueColumns().length() != actual.valueColumns().length())
+            return false;
+
+        for (int i = 0; i < expected.length(); i++) {
+            if (!expected.column(i).equals(actual.column(i)))
+                return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Schema descriptor factory method.
+     *
+     * @param ver Version.
+     * @param keyCols Key columns.
+     * @param valCols Value columns.
+     * @return Schema descriptor.
+     */
+    static SchemaDescriptor createDescriptor(int ver, List<Column> keyCols, List<Column> valCols) {
+        return new SchemaDescriptor(
+            ver,
+            keyCols.toArray(EMPTY_COLS_ARR),
+            valCols.toArray(EMPTY_COLS_ARR));
+    }
+}
diff --git a/modules/table/src/test/java/org/apache/ignite/table/impl/DummyInternalTableImpl.java b/modules/table/src/test/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java
similarity index 99%
rename from modules/table/src/test/java/org/apache/ignite/table/impl/DummyInternalTableImpl.java
rename to modules/table/src/test/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java
index 5e9a352..f41b025 100644
--- a/modules/table/src/test/java/org/apache/ignite/table/impl/DummyInternalTableImpl.java
+++ b/modules/table/src/test/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.table.impl;
+package org.apache.ignite.internal.table.impl;
 
 import java.util.Arrays;
 import java.util.Collection;
diff --git a/modules/table/src/test/java/org/apache/ignite/table/impl/DummySchemaManagerImpl.java b/modules/table/src/test/java/org/apache/ignite/internal/table/impl/DummySchemaManagerImpl.java
similarity index 97%
rename from modules/table/src/test/java/org/apache/ignite/table/impl/DummySchemaManagerImpl.java
rename to modules/table/src/test/java/org/apache/ignite/internal/table/impl/DummySchemaManagerImpl.java
index 01df2fe..3c19cf6 100644
--- a/modules/table/src/test/java/org/apache/ignite/table/impl/DummySchemaManagerImpl.java
+++ b/modules/table/src/test/java/org/apache/ignite/internal/table/impl/DummySchemaManagerImpl.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.ignite.table.impl;
+package org.apache.ignite.internal.table.impl;
 
 import org.apache.ignite.internal.schema.SchemaDescriptor;
 import org.apache.ignite.internal.table.TableSchemaManager;
diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/schema/SchemaManagerTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/schema/SchemaManagerTest.java
new file mode 100644
index 0000000..e056935
--- /dev/null
+++ b/modules/table/src/test/java/org/apache/ignite/internal/table/schema/SchemaManagerTest.java
@@ -0,0 +1,98 @@
+/*
+ * 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.table.schema;
+
+import org.apache.ignite.internal.schema.Column;
+import org.apache.ignite.internal.schema.SchemaDescriptor;
+import org.junit.jupiter.api.Test;
+
+import static org.apache.ignite.internal.schema.NativeType.BYTES;
+import static org.apache.ignite.internal.schema.NativeType.LONG;
+import static org.apache.ignite.internal.schema.NativeType.STRING;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Schema manager test.
+ */
+public class SchemaManagerTest {
+    /**
+     * Check schema registration.
+     */
+    @Test
+    void testSchemaRegistration() {
+        final SchemaDescriptor schemaV0 = new SchemaDescriptor(SchemaRegistry.INITIAL_SCHEMA_VERSION,
+            new Column[]{new Column("keyLongCol", LONG, true)},
+            new Column[]{new Column("valBytesCol", BYTES, true)});
+
+        final SchemaDescriptor schemaV1 = new SchemaDescriptor(1,
+            new Column[]{new Column("keyLongCol", LONG, true)},
+            new Column[]{new Column("valBytesCol", BYTES, true)});
+
+        final SchemaDescriptor schemaV2 = new SchemaDescriptor(1,
+            new Column[]{new Column("keyLongCol", LONG, true)},
+            new Column[] {
+                new Column("valBytesCol", BYTES, true),
+                new Column("valStringCol", STRING, true)
+            });
+
+        final SchemaDescriptor schemaV4 = new SchemaDescriptor(4,
+            new Column[]{new Column("keyLongCol", LONG, true)},
+            new Column[] {
+                new Column("valBytesCol", BYTES, true),
+                new Column("valStringCol", STRING, true)
+            });
+
+        SchemaRegistry mgr = new SchemaRegistry();
+
+        assertNull(mgr.schema(SchemaRegistry.INITIAL_SCHEMA_VERSION));
+        assertNull(mgr.schema(1));
+
+        // Register schema with initial version.
+        mgr.registerSchema(schemaV0);
+
+        assertEquals(1, mgr.schema().version());
+        assertTrue(TableSchemaManagerImpl.equalSchemas(schemaV0, mgr.schema(1)));
+        assertNull(mgr.schema(2));
+
+        // Register same schema with same version.
+        mgr.registerSchema(schemaV1);
+
+        assertEquals(1, mgr.schema().version());
+        assertTrue(TableSchemaManagerImpl.equalSchemas(schemaV1, mgr.schema(1)));
+        assertNull(mgr.schema(2));
+
+        // Register new schema with same version.
+        mgr.registerSchema(schemaV2);
+
+        assertEquals(2, mgr.schema().version());
+        assertTrue(TableSchemaManagerImpl.equalSchemas(schemaV1, mgr.schema(1)));
+        assertTrue(TableSchemaManagerImpl.equalSchemas(schemaV2, mgr.schema(2)));
+
+        // Version 3 missed.
+        assertThrows(SchemaRegistryException.class, () -> mgr.registerSchema(schemaV4));
+
+        assertEquals(2, mgr.schema().version());
+
+        assertTrue(TableSchemaManagerImpl.equalSchemas(schemaV2, mgr.schema(2)));
+        assertNull(mgr.schema(3));
+        assertNull(mgr.schema(4));
+    }
+}
diff --git a/modules/table/src/test/java/org/apache/ignite/table/Example.java b/modules/table/src/test/java/org/apache/ignite/table/Example.java
index 242a185..b83b75c 100644
--- a/modules/table/src/test/java/org/apache/ignite/table/Example.java
+++ b/modules/table/src/test/java/org/apache/ignite/table/Example.java
@@ -23,7 +23,7 @@ import java.util.List;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryObjects;
 import org.apache.ignite.internal.table.TableImpl;
-import org.apache.ignite.table.impl.DummyInternalTableImpl;
+import org.apache.ignite.internal.table.impl.DummyInternalTableImpl;
 import org.apache.ignite.table.mapper.Mappers;
 import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.params.ParameterizedTest;
diff --git a/modules/table/src/test/java/org/apache/ignite/table/KVViewOperationsTest.java b/modules/table/src/test/java/org/apache/ignite/table/KVViewOperationsTest.java
index 33f1650..a249a8a 100644
--- a/modules/table/src/test/java/org/apache/ignite/table/KVViewOperationsTest.java
+++ b/modules/table/src/test/java/org/apache/ignite/table/KVViewOperationsTest.java
@@ -21,8 +21,8 @@ import org.apache.ignite.internal.schema.Column;
 import org.apache.ignite.internal.schema.NativeType;
 import org.apache.ignite.internal.schema.SchemaDescriptor;
 import org.apache.ignite.internal.table.KVBinaryViewImpl;
-import org.apache.ignite.table.impl.DummyInternalTableImpl;
-import org.apache.ignite.table.impl.DummySchemaManagerImpl;
+import org.apache.ignite.internal.table.impl.DummyInternalTableImpl;
+import org.apache.ignite.internal.table.impl.DummySchemaManagerImpl;
 import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.Test;
 
diff --git a/modules/table/src/test/java/org/apache/ignite/table/TableBinaryViewOperationsTest.java b/modules/table/src/test/java/org/apache/ignite/table/TableBinaryViewOperationsTest.java
index 6b011b0..efbb2a3 100644
--- a/modules/table/src/test/java/org/apache/ignite/table/TableBinaryViewOperationsTest.java
+++ b/modules/table/src/test/java/org/apache/ignite/table/TableBinaryViewOperationsTest.java
@@ -21,8 +21,8 @@ import org.apache.ignite.internal.schema.Column;
 import org.apache.ignite.internal.schema.NativeType;
 import org.apache.ignite.internal.schema.SchemaDescriptor;
 import org.apache.ignite.internal.table.TableImpl;
-import org.apache.ignite.table.impl.DummyInternalTableImpl;
-import org.apache.ignite.table.impl.DummySchemaManagerImpl;
+import org.apache.ignite.internal.table.impl.DummyInternalTableImpl;
+import org.apache.ignite.internal.table.impl.DummySchemaManagerImpl;
 import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.Test;