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/10/27 09:42:09 UTC

[ignite-3] 04/04: Implement KV view operations.

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

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

commit 257cfd6fcf4d5e497d01531fecf69707e59b986c
Author: Andrew Mashenkov <an...@gmail.com>
AuthorDate: Wed Oct 27 12:40:01 2021 +0300

    Implement KV view operations.
---
 .../ignite/internal/table/KeyValueViewImpl.java    |  99 ++++---
 .../table/KeyValueBinaryViewOperationsTest.java    | 111 ++++---
 .../internal/table/KeyValueOperationsTest.java     | 328 ++++++++-------------
 3 files changed, 230 insertions(+), 308 deletions(-)

diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/KeyValueViewImpl.java b/modules/table/src/main/java/org/apache/ignite/internal/table/KeyValueViewImpl.java
index 4193198..90e8bbf 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/KeyValueViewImpl.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/KeyValueViewImpl.java
@@ -26,13 +26,11 @@ import java.util.Objects;
 import java.util.concurrent.CompletableFuture;
 import org.apache.ignite.internal.schema.BinaryRow;
 import org.apache.ignite.internal.schema.ByteBufferRow;
-import org.apache.ignite.internal.schema.SchemaDescriptor;
 import org.apache.ignite.internal.schema.SchemaRegistry;
 import org.apache.ignite.internal.schema.marshaller.KVSerializer;
 import org.apache.ignite.internal.schema.marshaller.SerializationException;
 import org.apache.ignite.internal.schema.marshaller.Serializer;
 import org.apache.ignite.internal.schema.marshaller.SerializerFactory;
-import org.apache.ignite.internal.schema.row.Row;
 import org.apache.ignite.lang.IgniteException;
 import org.apache.ignite.table.InvokeProcessor;
 import org.apache.ignite.table.KeyValueView;
@@ -75,19 +73,15 @@ public class KeyValueViewImpl<K, V> extends AbstractTableView implements KeyValu
 
     /** {@inheritDoc} */
     @Override public @NotNull CompletableFuture<V> getAsync(@NotNull K key) {
-        Objects.requireNonNull(key);
-
-        final KVSerializer<K, V> marsh = marshaller();
-
-        BinaryRow kRow = marsh.serialize(key, null); // Convert to portable format to pass TX/storage layer.
+        BinaryRow kRow = marshal(Objects.requireNonNull(key), null);
 
         return tbl.get(kRow, tx)
-            .thenApply(v -> v == null ? null : marsh.deserializeValue(v)); // row -> deserialized obj.
+            .thenApply(this::unmarshalValue); // row -> deserialized obj.
     }
 
     /** {@inheritDoc} */
     @Override public Map<K, V> getAll(@NotNull Collection<K> keys) {
-        throw new UnsupportedOperationException("Not implemented yet.");
+        return sync(getAllAsync(keys));
     }
 
     /** {@inheritDoc} */
@@ -97,12 +91,12 @@ public class KeyValueViewImpl<K, V> extends AbstractTableView implements KeyValu
 
     /** {@inheritDoc} */
     @Override public boolean contains(@NotNull K key) {
-        throw new UnsupportedOperationException("Not implemented yet.");
+        return get(key) != null;
     }
 
     /** {@inheritDoc} */
     @Override public CompletableFuture<Boolean> containsAsync(@NotNull K key) {
-        throw new UnsupportedOperationException("Not implemented yet.");
+        return getAsync(key).thenApply(Objects::nonNull);
     }
 
     /** {@inheritDoc} */
@@ -112,11 +106,7 @@ public class KeyValueViewImpl<K, V> extends AbstractTableView implements KeyValu
 
     /** {@inheritDoc} */
     @Override public @NotNull CompletableFuture<Void> putAsync(@NotNull K key, V val) {
-        Objects.requireNonNull(key);
-
-        final KVSerializer<K, V> marsh = marshaller();
-
-        BinaryRow kRow = marsh.serialize(key, val); // Convert to portable format to pass TX/storage layer.
+        BinaryRow kRow = marshal(Objects.requireNonNull(key), val);
 
         return tbl.upsert(kRow, tx).thenAccept(ignore -> {
         });
@@ -124,7 +114,7 @@ public class KeyValueViewImpl<K, V> extends AbstractTableView implements KeyValu
 
     /** {@inheritDoc} */
     @Override public void putAll(@NotNull Map<K, V> pairs) {
-        throw new UnsupportedOperationException("Not implemented yet.");
+        sync(putAllAsync(pairs));
     }
 
     /** {@inheritDoc} */
@@ -134,47 +124,55 @@ public class KeyValueViewImpl<K, V> extends AbstractTableView implements KeyValu
 
     /** {@inheritDoc} */
     @Override public V getAndPut(@NotNull K key, V val) {
-        throw new UnsupportedOperationException("Not implemented yet.");
+        return sync(getAndPutAsync(key, val));
     }
 
     /** {@inheritDoc} */
     @Override public @NotNull CompletableFuture<V> getAndPutAsync(@NotNull K key, V val) {
-        throw new UnsupportedOperationException("Not implemented yet.");
+        BinaryRow kRow = marshal(Objects.requireNonNull(key), val);
+
+        return tbl.getAndUpsert(kRow, tx).thenApply(this::unmarshalValue);
     }
 
     /** {@inheritDoc} */
     @Override public boolean putIfAbsent(@NotNull K key, V val) {
-        throw new UnsupportedOperationException("Not implemented yet.");
+        return sync(putIfAbsentAsync(key, val));
     }
 
     /** {@inheritDoc} */
     @Override public @NotNull CompletableFuture<Boolean> putIfAbsentAsync(@NotNull K key, V val) {
-        throw new UnsupportedOperationException("Not implemented yet.");
+        BinaryRow kRow = marshal(Objects.requireNonNull(key), val);
+
+        return tbl.insert(kRow, tx);
     }
 
     /** {@inheritDoc} */
     @Override public boolean remove(@NotNull K key) {
-        throw new UnsupportedOperationException("Not implemented yet.");
+        return sync(removeAsync(key));
     }
 
     /** {@inheritDoc} */
     @Override public @NotNull CompletableFuture<Boolean> removeAsync(@NotNull K key) {
-        throw new UnsupportedOperationException("Not implemented yet.");
+        BinaryRow kRow = marshal(Objects.requireNonNull(key), null);
+
+        return tbl.delete(kRow, tx);
     }
 
     /** {@inheritDoc} */
     @Override public boolean remove(@NotNull K key, @NotNull V val) {
-        throw new UnsupportedOperationException("Not implemented yet.");
+        return sync(removeAsync(key, val));
     }
 
     /** {@inheritDoc} */
     @Override public @NotNull CompletableFuture<Boolean> removeAsync(@NotNull K key, @NotNull V val) {
-        throw new UnsupportedOperationException("Not implemented yet.");
+        BinaryRow kRow = marshal(Objects.requireNonNull(key), val);
+
+        return tbl.deleteExact(kRow, tx);
     }
 
     /** {@inheritDoc} */
     @Override public Collection<K> removeAll(@NotNull Collection<K> keys) {
-        throw new UnsupportedOperationException("Not implemented yet.");
+        return sync(removeAllAsync(keys));
     }
 
     /** {@inheritDoc} */
@@ -184,46 +182,56 @@ public class KeyValueViewImpl<K, V> extends AbstractTableView implements KeyValu
 
     /** {@inheritDoc} */
     @Override public V getAndRemove(@NotNull K key) {
-        throw new UnsupportedOperationException("Not implemented yet.");
+        return sync(getAndRemoveAsync(key));
     }
 
     /** {@inheritDoc} */
     @Override public @NotNull CompletableFuture<V> getAndRemoveAsync(@NotNull K key) {
-        throw new UnsupportedOperationException("Not implemented yet.");
+        BinaryRow kRow = marshal(Objects.requireNonNull(key), null);
+
+        return tbl.getAndDelete(kRow, tx).thenApply(this::unmarshalValue);
     }
 
     /** {@inheritDoc} */
     @Override public boolean replace(@NotNull K key, V val) {
-        throw new UnsupportedOperationException("Not implemented yet.");
+        return sync(replaceAsync(key, val));
     }
 
     /** {@inheritDoc} */
     @Override public @NotNull CompletableFuture<Boolean> replaceAsync(@NotNull K key, V val) {
-        throw new UnsupportedOperationException("Not implemented yet.");
+        BinaryRow row = marshal(key, val);
+
+        return tbl.replace(row, tx);
     }
 
     /** {@inheritDoc} */
     @Override public boolean replace(@NotNull K key, V oldVal, V newVal) {
-        throw new UnsupportedOperationException("Not implemented yet.");
+        return sync(replaceAsync(key, oldVal, newVal));
     }
 
     /** {@inheritDoc} */
     @Override public @NotNull CompletableFuture<Boolean> replaceAsync(@NotNull K key, V oldVal, V newVal) {
-        throw new UnsupportedOperationException("Not implemented yet.");
+        BinaryRow oldRow = marshal(key, oldVal);
+        BinaryRow newRow = marshal(key, newVal);
+
+        return tbl.replace(oldRow, newRow, tx);
     }
 
     /** {@inheritDoc} */
     @Override public V getAndReplace(@NotNull K key, V val) {
-        throw new UnsupportedOperationException("Not implemented yet.");
+        return sync(getAndReplaceAsync(key, val));
     }
 
     /** {@inheritDoc} */
     @Override public @NotNull CompletableFuture<V> getAndReplaceAsync(@NotNull K key, V val) {
-        throw new UnsupportedOperationException("Not implemented yet.");
+        BinaryRow row = marshal(key, val);
+
+        return tbl.getAndReplace(row, tx).thenApply(this::unmarshalValue);
     }
 
     /** {@inheritDoc} */
-    @Override public <R extends Serializable> R invoke(@NotNull K key, InvokeProcessor<K, V, R> proc, Serializable... args) {
+    @Override
+    public <R extends Serializable> R invoke(@NotNull K key, InvokeProcessor<K, V, R> proc, Serializable... args) {
         throw new UnsupportedOperationException("Not implemented yet.");
     }
 
@@ -259,13 +267,15 @@ public class KeyValueViewImpl<K, V> extends AbstractTableView implements KeyValu
     }
 
     /**
+     * @param schemaVersion Schema version.
      * @return Marshaller.
      */
-    private KVSerializer<K, V> marshaller() {
+    private KVSerializer<K, V> marshaller(int schemaVersion) {
         SerializerFactory factory = SerializerFactory.createJavaSerializerFactory();
 
+        // TODO: Cache marshaller for schema or upgrade row?
         return new KVSerializer<K, V>() {
-            Serializer s = factory.create(schemaReg.schema(), keyMapper.getType(), valueMapper.getType());
+            Serializer s = factory.create(schemaReg.schema(schemaVersion), keyMapper.getType(), valueMapper.getType());
 
             @Override public BinaryRow serialize(@NotNull K key, V val) {
                 try {
@@ -293,16 +303,13 @@ public class KeyValueViewImpl<K, V> extends AbstractTableView implements KeyValu
         };
     }
 
-    /**
-     * @param row Binary row.
-     * @return Schema-aware row.
-     */
-    private Row wrap(BinaryRow row) {
-        if (row == null)
-            return null;
+    private V unmarshalValue(BinaryRow v) {
+        return v == null ? null : marshaller(v.schemaVersion()).deserializeValue(v);
+    }
 
-        final SchemaDescriptor rowSchema = schemaReg.schema(row.schemaVersion()); // Get a schema for row.
+    private BinaryRow marshal(@NotNull K key, V o) {
+        final KVSerializer<K, V> marsh = marshaller(schemaReg.lastSchemaVersion());
 
-        return new Row(rowSchema, row);
+        return marsh.serialize(key, o);
     }
 }
diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/KeyValueBinaryViewOperationsTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/KeyValueBinaryViewOperationsTest.java
index ef4a133..5871469 100644
--- a/modules/table/src/test/java/org/apache/ignite/internal/table/KeyValueBinaryViewOperationsTest.java
+++ b/modules/table/src/test/java/org/apache/ignite/internal/table/KeyValueBinaryViewOperationsTest.java
@@ -41,19 +41,28 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
  * TODO: IGNITE-14487 Check key fields in Tuple is ignored for value or exception is thrown?
  */
 public class KeyValueBinaryViewOperationsTest {
+    /** Simple schema. */
+    private SchemaDescriptor schema = new SchemaDescriptor(
+        1,
+        new Column[]{new Column("id", NativeTypes.INT64, false)},
+        new Column[]{new Column("val", NativeTypes.INT64, false)}
+    );
+
+    /**
+     * Creates table view.
+     *
+     * @return Table KV binary view.
+     */
+    private KeyValueView<Tuple, Tuple> tableView() {
+        return new KeyValueBinaryViewImpl(new DummyInternalTableImpl(), new DummySchemaManagerImpl(schema), null, null);
+    }
+
     /**
      *
      */
     @Test
     public void put() {
-        SchemaDescriptor schema = new SchemaDescriptor(
-            1,
-            new Column[] {new Column("id", NativeTypes.INT64, false)},
-            new Column[] {new Column("val", NativeTypes.INT64, false)}
-        );
-
-        KeyValueView<Tuple, Tuple> tbl =
-            new KeyValueBinaryViewImpl(new DummyInternalTableImpl(), new DummySchemaManagerImpl(schema), null, null);
+        KeyValueView<Tuple, Tuple> tbl = tableView();
 
         final Tuple key = Tuple.create().set("id", 1L);
         final Tuple val = Tuple.create().set("val", 11L);
@@ -89,14 +98,7 @@ public class KeyValueBinaryViewOperationsTest {
      */
     @Test
     public void putIfAbsent() {
-        SchemaDescriptor schema = new SchemaDescriptor(
-            1,
-            new Column[] {new Column("id", NativeTypes.INT64, false)},
-            new Column[] {new Column("val", NativeTypes.INT64, false)}
-        );
-
-        KeyValueView<Tuple, Tuple> tbl =
-            new KeyValueBinaryViewImpl(new DummyInternalTableImpl(), new DummySchemaManagerImpl(schema), null, null);
+        KeyValueView<Tuple, Tuple> tbl = tableView();
 
         final Tuple key = Tuple.create().set("id", 1L);
         final Tuple val = Tuple.create().set("val", 11L);
@@ -122,14 +124,7 @@ public class KeyValueBinaryViewOperationsTest {
      */
     @Test
     public void getAndPut() {
-        SchemaDescriptor schema = new SchemaDescriptor(
-            1,
-            new Column[] {new Column("id", NativeTypes.INT64, false)},
-            new Column[] {new Column("val", NativeTypes.INT64, false)}
-        );
-
-        KeyValueView<Tuple, Tuple> tbl =
-            new KeyValueBinaryViewImpl(new DummyInternalTableImpl(), new DummySchemaManagerImpl(schema), null, null);
+        KeyValueView<Tuple, Tuple> tbl = tableView();
 
         final Tuple key = Tuple.create().set("id", 1L);
         final Tuple val = Tuple.create().set("val", 11L);
@@ -155,15 +150,40 @@ public class KeyValueBinaryViewOperationsTest {
      *
      */
     @Test
-    public void remove() {
-        SchemaDescriptor schema = new SchemaDescriptor(
-            1,
-            new Column[] {new Column("id", NativeTypes.INT64, false)},
-            new Column[] {new Column("val", NativeTypes.INT64, false)}
-        );
+    public void contains() {
+        KeyValueView<Tuple, Tuple> tbl = tableView();
+
+        final Tuple key = Tuple.create().set("id", 1L);
+        final Tuple val = Tuple.create().set("val", 11L);
+        final Tuple val2 = Tuple.create().set("val", 22L);
+
+        // Not-existed value.
+        assertFalse(tbl.contains(key));
+
+        // Put KV pair.
+        tbl.put(key, val);
+        assertTrue(tbl.contains(Tuple.create().set("id", 1L)));
+
+        // Delete key.
+        assertTrue(tbl.remove(key));
+        assertFalse(tbl.contains(Tuple.create().set("id", 1L)));
+
+        // Put KV pair.
+        tbl.put(key, val2);
+        assertTrue(tbl.contains(Tuple.create().set("id", 1L)));
+
+        // Non-existed key.
+        assertFalse(tbl.contains(Tuple.create().set("id", 2L)));
+        tbl.remove(Tuple.create().set("id", 2L));
+        assertFalse(tbl.contains(Tuple.create().set("id", 2L)));
+    }
 
-        KeyValueView<Tuple, Tuple> tbl =
-            new KeyValueBinaryViewImpl(new DummyInternalTableImpl(), new DummySchemaManagerImpl(schema), null, null);
+    /**
+     *
+     */
+    @Test
+    public void remove() {
+        KeyValueView<Tuple, Tuple> tbl = tableView();
 
         final Tuple key = Tuple.create().set("id", 1L);
         final Tuple key2 = Tuple.create().set("id", 2L);
@@ -199,14 +219,7 @@ public class KeyValueBinaryViewOperationsTest {
      */
     @Test
     public void removeExact() {
-        SchemaDescriptor schema = new SchemaDescriptor(
-            1,
-            new Column[] {new Column("id", NativeTypes.INT64, false)},
-            new Column[] {new Column("val", NativeTypes.INT64, false)}
-        );
-
-        final KeyValueView<Tuple, Tuple> tbl =
-            new KeyValueBinaryViewImpl(new DummyInternalTableImpl(), new DummySchemaManagerImpl(schema), null, null);
+        final KeyValueView<Tuple, Tuple> tbl = tableView();
 
         final Tuple key = Tuple.create().set("id", 1L);
         final Tuple key2 = Tuple.create().set("id", 2L);
@@ -254,14 +267,7 @@ public class KeyValueBinaryViewOperationsTest {
      */
     @Test
     public void replace() {
-        SchemaDescriptor schema = new SchemaDescriptor(
-            1,
-            new Column[] {new Column("id", NativeTypes.INT64, false)},
-            new Column[] {new Column("val", NativeTypes.INT64, false)}
-        );
-
-        KeyValueView<Tuple, Tuple> tbl =
-            new KeyValueBinaryViewImpl(new DummyInternalTableImpl(), new DummySchemaManagerImpl(schema), null, null);
+        KeyValueView<Tuple, Tuple> tbl = tableView();
 
         final Tuple key = Tuple.create().set("id", 1L);
         final Tuple key2 = Tuple.create().set("id", 2L);
@@ -300,14 +306,7 @@ public class KeyValueBinaryViewOperationsTest {
      */
     @Test
     public void replaceExact() {
-        SchemaDescriptor schema = new SchemaDescriptor(
-            1,
-            new Column[] {new Column("id", NativeTypes.INT64, false)},
-            new Column[] {new Column("val", NativeTypes.INT64, false)}
-        );
-
-        KeyValueView<Tuple, Tuple> tbl =
-            new KeyValueBinaryViewImpl(new DummyInternalTableImpl(), new DummySchemaManagerImpl(schema), null, null);
+        KeyValueView<Tuple, Tuple> tbl = tableView();
 
         final Tuple key = Tuple.create().set("id", 1L);
         final Tuple key2 = Tuple.create().set("id", 2L);
diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/KeyValueOperationsTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/KeyValueOperationsTest.java
index 12fc840..e5c70b4 100644
--- a/modules/table/src/test/java/org/apache/ignite/internal/table/KeyValueOperationsTest.java
+++ b/modules/table/src/test/java/org/apache/ignite/internal/table/KeyValueOperationsTest.java
@@ -23,7 +23,6 @@ import org.apache.ignite.internal.schema.SchemaDescriptor;
 import org.apache.ignite.internal.table.impl.DummyInternalTableImpl;
 import org.apache.ignite.internal.table.impl.DummySchemaManagerImpl;
 import org.apache.ignite.table.KeyValueView;
-import org.apache.ignite.table.Tuple;
 import org.apache.ignite.table.mapper.Mapper;
 import org.junit.jupiter.api.Test;
 
@@ -40,25 +39,35 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
  * TODO: IGNITE-14487 Add async operations tests.
  */
 public class KeyValueOperationsTest {
+    /** Default mapper. */
+    private final Mapper<Long> mapper = new Mapper<>() {
+        @Override public Class<Long> getType() {
+            return Long.class;
+        }
+    };
+
+    /** Simple schema. */
+    private SchemaDescriptor schema = new SchemaDescriptor(
+        1,
+        new Column[]{new Column("id", NativeTypes.INT64, false)},
+        new Column[]{new Column("val", NativeTypes.INT64, false)}
+    );
+
+    /**
+     * Creates table view.
+     *
+     * @return Table KV-view.
+     */
+    private KeyValueView<Long, Long> tableView() {
+        return new KeyValueViewImpl<>(new DummyInternalTableImpl(), new DummySchemaManagerImpl(schema), mapper, mapper, null);
+    }
+
     /**
      *
      */
     @Test
     public void put() {
-        SchemaDescriptor schema = new SchemaDescriptor(
-            1,
-            new Column[] {new Column("id", NativeTypes.INT64, false)},
-            new Column[] {new Column("val", NativeTypes.INT64, false)}
-        );
-
-        Mapper<Long> mapper = new Mapper<>() {
-            @Override public Class<Long> getType() {
-                return Long.class;
-            }
-        };
-
-        KeyValueView<Long, Long> tbl =
-            new KeyValueViewImpl<>(new DummyInternalTableImpl(), new DummySchemaManagerImpl(schema), mapper, mapper, null);
+        KeyValueView<Long, Long> tbl = tableView();
 
         assertNull(tbl.get(1L));
 
@@ -72,7 +81,7 @@ public class KeyValueOperationsTest {
         tbl.put(1L, 22L);
 
         assertEquals(22L, tbl.get(1L));
-        assertEquals( 22L, tbl.get(1L));
+        assertEquals(22L, tbl.get(1L));
 
         // Remove KV pair.
         tbl.put(1L, null);
@@ -89,32 +98,19 @@ public class KeyValueOperationsTest {
      */
     @Test
     public void putIfAbsent() {
-        SchemaDescriptor schema = new SchemaDescriptor(
-            1,
-            new Column[] {new Column("id", NativeTypes.INT64, false)},
-            new Column[] {new Column("val", NativeTypes.INT64, false)}
-        );
+        KeyValueView<Long, Long> tbl = tableView();
 
-        KeyValueView<Tuple, Tuple> tbl =
-            new KeyValueBinaryViewImpl(new DummyInternalTableImpl(), new DummySchemaManagerImpl(schema), null, null);
-
-        final Tuple key = Tuple.create().set("id", 1L);
-        final Tuple val = Tuple.create().set("val", 11L);
-        final Tuple val2 = Tuple.create().set("val", 22L);
-
-        assertNull(tbl.get(key));
+        assertNull(tbl.get(1L));
 
         // Insert new KV pair.
-        assertTrue(tbl.putIfAbsent(key, val));
+        assertTrue(tbl.putIfAbsent(1L, 11L));
 
-        assertEqualsValues(schema, val, tbl.get(key));
-        assertEqualsValues(schema, val, tbl.get(Tuple.create().set("id", 1L)));
+        assertEquals(11L, tbl.get(1L));
 
         // Update KV pair.
-        assertFalse(tbl.putIfAbsent(key, val2));
+        assertFalse(tbl.putIfAbsent(1L, 22L));
 
-        assertEqualsValues(schema, val, tbl.get(key));
-        assertEqualsValues(schema, val, tbl.get(Tuple.create().set("id", 1L)));
+        assertEquals(11L, tbl.get(1L));
     }
 
     /**
@@ -122,76 +118,77 @@ public class KeyValueOperationsTest {
      */
     @Test
     public void getAndPut() {
-        SchemaDescriptor schema = new SchemaDescriptor(
-            1,
-            new Column[] {new Column("id", NativeTypes.INT64, false)},
-            new Column[] {new Column("val", NativeTypes.INT64, false)}
-        );
-
-        KeyValueView<Tuple, Tuple> tbl =
-            new KeyValueBinaryViewImpl(new DummyInternalTableImpl(), new DummySchemaManagerImpl(schema), null, null);
+        KeyValueView<Long, Long> tbl = tableView();
 
-        final Tuple key = Tuple.create().set("id", 1L);
-        final Tuple val = Tuple.create().set("val", 11L);
-        final Tuple val2 = Tuple.create().set("val", 22L);
-        final Tuple val3 = Tuple.create().set("val", 33L);
-
-        assertNull(tbl.get(key));
+        assertNull(tbl.get(1L));
 
         // Insert new tuple.
-        assertNull(tbl.getAndPut(key, val));
+        assertNull(tbl.getAndPut(1L, 11L));
 
-        assertEqualsValues(schema, val, tbl.get(key));
-        assertEqualsValues(schema, val, tbl.get(Tuple.create().set("id", 1L)));
+        assertEquals(11L, tbl.get(1L));
 
-        assertEqualsValues(schema, val, tbl.getAndPut(key, val2));
-        assertEqualsValues(schema, val2, tbl.getAndPut(key, Tuple.create().set("val", 33L)));
+        assertEquals(11L, tbl.getAndPut(1L, 22L));
+        assertEquals(22L, tbl.getAndPut(1L, 33L));
 
-        assertEqualsValues(schema, val3, tbl.get(key));
-        assertNull(tbl.get(Tuple.create().set("id", 2L)));
+        assertEquals(33L, tbl.get(1L));
     }
 
     /**
      *
      */
     @Test
-    public void remove() {
-        SchemaDescriptor schema = new SchemaDescriptor(
-            1,
-            new Column[] {new Column("id", NativeTypes.INT64, false)},
-            new Column[] {new Column("val", NativeTypes.INT64, false)}
-        );
+    public void contains() {
+        KeyValueView<Long, Long> tbl = tableView();
 
-        KeyValueView<Tuple, Tuple> tbl =
-            new KeyValueBinaryViewImpl(new DummyInternalTableImpl(), new DummySchemaManagerImpl(schema), null, null);
+        // Not-existed value.
+        assertFalse(tbl.contains(1L));
 
-        final Tuple key = Tuple.create().set("id", 1L);
-        final Tuple key2 = Tuple.create().set("id", 2L);
-        final Tuple val = Tuple.create().set("val", 11L);
-        final Tuple val2 = Tuple.create().set("val", 22L);
+        // Put KV pair.
+        tbl.put(1L, 11L);
+        assertTrue(tbl.contains(1L));
+
+        // Delete key.
+        assertTrue(tbl.remove(1L));
+        assertFalse(tbl.contains(1L));
+
+        // Put KV pair.
+        tbl.put(1L, 22L);
+        assertTrue(tbl.contains(1L));
+
+        // Delete key.
+        tbl.remove(2L);
+        assertFalse(tbl.contains(2L));
+    }
+
+    /**
+     *
+     */
+    @Test
+    public void remove() {
+        KeyValueView<Long, Long> tbl = tableView();
 
         // Put KV pair.
-        tbl.put(key, val);
+        tbl.put(1L, 11L);
 
         // Delete existed key.
-        assertEqualsValues(schema, val, tbl.get(key));
-        assertTrue(tbl.remove(key));
-        assertNull(tbl.get(key));
+        assertEquals(11L, tbl.get(1L));
+        assertTrue(tbl.remove(1L));
+        assertNull(tbl.get(1L));
 
         // Delete already deleted key.
-        assertFalse(tbl.remove(key));
+        assertFalse(tbl.remove(1L));
 
         // Put KV pair.
-        tbl.put(key, val2);
-        assertEqualsValues(schema, val2, tbl.get(key));
+        tbl.put(1L, 22L);
+        assertEquals(22L, tbl.get(1L));
 
         // Delete existed key.
-        assertTrue(tbl.remove(Tuple.create().set("id", 1L)));
-        assertNull(tbl.get(key));
+        assertTrue(tbl.remove(1L));
+        assertNull(tbl.get(1L));
 
         // Delete not existed key.
-        assertNull(tbl.get(key2));
-        assertFalse(tbl.remove(key2));
+        assertNull(tbl.get(2L));
+        assertFalse(tbl.remove(2L));
     }
 
     /**
@@ -199,54 +196,42 @@ public class KeyValueOperationsTest {
      */
     @Test
     public void removeExact() {
-        SchemaDescriptor schema = new SchemaDescriptor(
-            1,
-            new Column[] {new Column("id", NativeTypes.INT64, false)},
-            new Column[] {new Column("val", NativeTypes.INT64, false)}
-        );
-
-        final KeyValueView<Tuple, Tuple> tbl =
-            new KeyValueBinaryViewImpl(new DummyInternalTableImpl(), new DummySchemaManagerImpl(schema), null, null);
-
-        final Tuple key = Tuple.create().set("id", 1L);
-        final Tuple key2 = Tuple.create().set("id", 2L);
-        final Tuple val = Tuple.create().set("val", 11L);
-        final Tuple val2 = Tuple.create().set("val", 22L);
+        KeyValueView<Long, Long> tbl = tableView();
 
         // Put KV pair.
-        tbl.put(key, val);
-        assertEqualsValues(schema, val, tbl.get(key));
+        tbl.put(1L, 11L);
+        assertEquals(11L, tbl.get(1L));
 
         // Fails to delete KV pair with unexpected value.
-        assertFalse(tbl.remove(key, val2));
-        assertEqualsValues(schema, val, tbl.get(key));
+        assertFalse(tbl.remove(1L, 22L));
+        assertEquals(11L, tbl.get(1L));
 
         // Delete KV pair with expected value.
-        assertTrue(tbl.remove(key, val));
-        assertNull(tbl.get(key));
+        assertTrue(tbl.remove(1L, 11L));
+        assertNull(tbl.get(1L));
 
         // Once again.
-        assertFalse(tbl.remove(key, val));
-        assertNull(tbl.get(key));
+        assertFalse(tbl.remove(1L, 11L));
+        assertNull(tbl.get(1L));
 
         // Try to remove non-existed key.
-        assertThrows(Exception.class, () -> tbl.remove(key, null));
-        assertNull(tbl.get(key));
+        assertThrows(Exception.class, () -> tbl.remove(1L, null));
+        assertNull(tbl.get(1L));
 
         // Put KV pair.
-        tbl.put(key, val2);
-        assertEqualsValues(schema, val2, tbl.get(key));
+        tbl.put(1L, 22L);
+        assertEquals(22L, tbl.get(1L));
 
         // Check null value ignored.
-        assertThrows(Exception.class, () -> tbl.remove(key, null));
-        assertEqualsValues(schema, val2, tbl.get(key));
+        assertThrows(Exception.class, () -> tbl.remove(1L, null));
+        assertEquals(22L, tbl.get(1L));
 
         // Delete KV pair with expected value.
-        assertTrue(tbl.remove(key, val2));
-        assertNull(tbl.get(key));
+        assertTrue(tbl.remove(1L, 22L));
+        assertNull(tbl.get(1L));
 
-        assertFalse(tbl.remove(key2, val2));
-        assertNull(tbl.get(key2));
+        assertFalse(tbl.remove(2L, 22L));
+        assertNull(tbl.get(2L));
     }
 
     /**
@@ -254,45 +239,32 @@ public class KeyValueOperationsTest {
      */
     @Test
     public void replace() {
-        SchemaDescriptor schema = new SchemaDescriptor(
-            1,
-            new Column[] {new Column("id", NativeTypes.INT64, false)},
-            new Column[] {new Column("val", NativeTypes.INT64, false)}
-        );
-
-        KeyValueView<Tuple, Tuple> tbl =
-            new KeyValueBinaryViewImpl(new DummyInternalTableImpl(), new DummySchemaManagerImpl(schema), null, null);
-
-        final Tuple key = Tuple.create().set("id", 1L);
-        final Tuple key2 = Tuple.create().set("id", 2L);
-        final Tuple val = Tuple.create().set("val", 11L);
-        final Tuple val2 = Tuple.create().set("val", 22L);
-        final Tuple val3 = Tuple.create().set("val", 33L);
+        KeyValueView<Long, Long> tbl = tableView();
 
         // Ignore replace operation for non-existed KV pair.
-        assertFalse(tbl.replace(key, val));
-        assertNull(tbl.get(key));
+        assertFalse(tbl.replace(1L, 11L));
+        assertNull(tbl.get(1L));
 
-        tbl.put(key, val);
+        tbl.put(1L, 11L);
 
         // Replace existed KV pair.
-        assertTrue(tbl.replace(key, val2));
-        assertEqualsValues(schema, val2, tbl.get(key));
+        assertTrue(tbl.replace(1L, 22L));
+        assertEquals(22L, tbl.get(1L));
 
         // Remove existed KV pair.
-        assertTrue(tbl.replace(key, null));
-        assertNull(tbl.get(key));
+        assertTrue(tbl.replace(1L, null));
+        assertNull(tbl.get(1L));
 
         // Ignore replace operation for non-existed KV pair.
-        assertFalse(tbl.replace(key, val3));
-        assertNull(tbl.get(key));
+        assertFalse(tbl.replace(1L, 33L));
+        assertNull(tbl.get(1L));
 
-        tbl.put(key, val3);
-        assertEqualsValues(schema, val3, tbl.get(key));
+        tbl.put(1L, 33L);
+        assertEquals(33L, tbl.get(1L));
 
         // Remove non-existed KV pair.
-        assertFalse(tbl.replace(key2, null));
-        assertNull(tbl.get(key2));
+        assertFalse(tbl.replace(2L, null));
+        assertNull(tbl.get(2L));
     }
 
     /**
@@ -300,86 +272,30 @@ public class KeyValueOperationsTest {
      */
     @Test
     public void replaceExact() {
-        SchemaDescriptor schema = new SchemaDescriptor(
-            1,
-            new Column[] {new Column("id", NativeTypes.INT64, false)},
-            new Column[] {new Column("val", NativeTypes.INT64, false)}
-        );
-
-        KeyValueView<Tuple, Tuple> tbl =
-            new KeyValueBinaryViewImpl(new DummyInternalTableImpl(), new DummySchemaManagerImpl(schema), null, null);
-
-        final Tuple key = Tuple.create().set("id", 1L);
-        final Tuple key2 = Tuple.create().set("id", 2L);
-        final Tuple val = Tuple.create().set("val", 11L);
-        final Tuple val2 = Tuple.create().set("val", 22L);
-        final Tuple val3 = Tuple.create().set("val", 33L);
+        KeyValueView<Long, Long> tbl = tableView();
 
         // Insert KV pair.
-        assertTrue(tbl.replace(key, null, val));
-        assertEqualsValues(schema, val, tbl.get(key));
-        assertNull(tbl.get(key2));
+        assertTrue(tbl.replace(1L, null, 11L));
+        assertEquals(11L, tbl.get(1L));
+        assertNull(tbl.get(2L));
 
         // Ignore replace operation for non-existed KV pair.
-        assertFalse(tbl.replace(key2, val, val2));
-        assertNull(tbl.get(key2));
+        assertFalse(tbl.replace(2L, 11L, 22L));
+        assertNull(tbl.get(2L));
 
         // Replace existed KV pair.
-        assertTrue(tbl.replace(key, val, val2));
-        assertEqualsValues(schema, val2, tbl.get(key));
+        assertTrue(tbl.replace(1L, 11L, 22L));
+        assertEquals(22L, tbl.get(1L));
 
         // Remove existed KV pair.
-        assertTrue(tbl.replace(key, val2, null));
-        assertNull(tbl.get(key));
+        assertTrue(tbl.replace(1L, 22L, null));
+        assertNull(tbl.get(1L));
 
         // Insert KV pair.
-        assertTrue(tbl.replace(key, null, val3));
-        assertEqualsValues(schema, val3, tbl.get(key));
+        assertTrue(tbl.replace(1L, null, 33L));
+        assertEquals(33L, tbl.get(1L));
 
         // Remove non-existed KV pair.
-        assertTrue(tbl.replace(key2, null, null));
-    }
-
-    /**
-     * Check key columns equality.
-     *
-     * @param schema Schema.
-     * @param expected Expected tuple.
-     * @param actual Actual tuple.
-     */
-    void assertEqualsKeys(SchemaDescriptor schema, Tuple expected, Tuple actual) {
-        int nonNullKey = 0;
-
-        for (int i = 0; i < schema.keyColumns().length(); i++) {
-            final Column col = schema.keyColumns().column(i);
-
-            final Object val1 = expected.value(col.name());
-            final Object val2 = actual.value(col.name());
-
-            assertEquals(val1, val2, "Value columns equality check failed: colIdx=" + col.schemaIndex());
-
-            if (schema.isKeyColumn(i) && val1 != null)
-                nonNullKey++;
-        }
-
-        assertTrue(nonNullKey > 0, "At least one non-null key column must exist.");
-    }
-
-    /**
-     * Check value columns equality.
-     *
-     * @param schema Schema.
-     * @param expected Expected tuple.
-     * @param actual Actual tuple.
-     */
-    void assertEqualsValues(SchemaDescriptor schema, Tuple expected, Tuple actual) {
-        for (int i = 0; i < schema.valueColumns().length(); i++) {
-            final Column col = schema.valueColumns().column(i);
-
-            final Object val1 = expected.value(col.name());
-            final Object val2 = actual.value(col.name());
-
-            assertEquals(val1, val2, "Key columns equality check failed: colIdx=" + col.schemaIndex());
-        }
+        assertTrue(tbl.replace(2L, null, null));
     }
 }