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/08/30 18:01:40 UTC

[ignite-3] branch main updated: IGNITE-15163 Support BitSet, Number, Decimal and Temporal types (#296)

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

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


The following commit(s) were added to refs/heads/main by this push:
     new fbf55e2  IGNITE-15163 Support BitSet, Number, Decimal and Temporal types (#296)
fbf55e2 is described below

commit fbf55e2fd30bc6f1d4dc6e813a4899ba8c958817
Author: Andrew V. Mashenkov <AM...@users.noreply.github.com>
AuthorDate: Mon Aug 30 21:01:35 2021 +0300

    IGNITE-15163 Support BitSet, Number, Decimal and Temporal types (#296)
---
 .../apache/ignite/client/proto/ClientDataType.java |   3 +
 .../ignite/client/proto/ClientMessagePacker.java   | 137 ++++++++++++++---
 .../ignite/client/proto/ClientMessageUnpacker.java | 131 ++++++++++++++--
 .../ignite/client/proto/ClientMsgPackType.java     |   6 +
 .../proto/ClientMessagePackerUnpackerTest.java     | 166 +++++++++++++++++++++
 .../handler/requests/table/ClientTableCommon.java  |   8 +
 modules/schema/pom.xml                             |   4 +-
 .../apache/ignite/internal/schema/TestUtils.java   |  10 +-
 8 files changed, 421 insertions(+), 44 deletions(-)

diff --git a/modules/client-common/src/main/java/org/apache/ignite/client/proto/ClientDataType.java b/modules/client-common/src/main/java/org/apache/ignite/client/proto/ClientDataType.java
index dbda15f..06cbf10 100644
--- a/modules/client-common/src/main/java/org/apache/ignite/client/proto/ClientDataType.java
+++ b/modules/client-common/src/main/java/org/apache/ignite/client/proto/ClientDataType.java
@@ -65,4 +65,7 @@ public class ClientDataType {
 
     /** Timestamp. */
     public static final int TIMESTAMP = 15;
+
+    /** Number. */
+    public static final int NUMBER = 16;
 }
diff --git a/modules/client-common/src/main/java/org/apache/ignite/client/proto/ClientMessagePacker.java b/modules/client-common/src/main/java/org/apache/ignite/client/proto/ClientMessagePacker.java
index 8b3254d..7f239f1 100644
--- a/modules/client-common/src/main/java/org/apache/ignite/client/proto/ClientMessagePacker.java
+++ b/modules/client-common/src/main/java/org/apache/ignite/client/proto/ClientMessagePacker.java
@@ -320,7 +320,7 @@ public class ClientMessagePacker extends MessagePacker {
         bb.putLong(val.getMostSignificantBits());
         bb.putLong(val.getLeastSignificantBits());
 
-        writePayload(bytes);
+        addPayload(bytes);
 
         return this;
     }
@@ -330,12 +330,37 @@ public class ClientMessagePacker extends MessagePacker {
      *
      * @param val Decimal value.
      * @return This instance.
-     * @throws UnsupportedOperationException Not supported.
      */
     public ClientMessagePacker packDecimal(BigDecimal val) {
         assert !closed : "Packer is closed";
 
-        throw new UnsupportedOperationException("TODO: IGNITE-15163");
+        // TODO: Pack directly to ByteBuf without allocating IGNITE-15234.
+        byte[] unscaledValue = val.unscaledValue().toByteArray();
+
+        packExtensionTypeHeader(ClientMsgPackType.DECIMAL, 4 + unscaledValue.length); // Scale length + data length
+
+        addPayload(ByteBuffer.wrap(new byte[4]).putInt(val.scale()).array());
+        addPayload(unscaledValue);
+
+        return this;
+    }
+
+    /**
+     * Writes a decimal.
+     *
+     * @param val Decimal value.
+     * @return This instance.
+     */
+    public ClientMessagePacker packNumber(BigInteger val) {
+        assert !closed : "Packer is closed";
+
+        byte[] data = val.toByteArray();
+
+        packExtensionTypeHeader(ClientMsgPackType.NUMBER, data.length);
+
+        addPayload(data);
+
+        return this;
     }
 
     /**
@@ -343,12 +368,18 @@ public class ClientMessagePacker extends MessagePacker {
      *
      * @param val Bit set value.
      * @return This instance.
-     * @throws UnsupportedOperationException Not supported.
      */
     public ClientMessagePacker packBitSet(BitSet val) {
         assert !closed : "Packer is closed";
 
-        throw new UnsupportedOperationException("TODO: IGNITE-15163");
+        // TODO: Pack directly to ByteBuf without allocating IGNITE-15234.
+        byte[] data = val.toByteArray();
+
+        packExtensionTypeHeader(ClientMsgPackType.BITMASK, data.length);
+
+        addPayload(data);
+
+        return this;
     }
 
     /**
@@ -356,12 +387,23 @@ public class ClientMessagePacker extends MessagePacker {
      *
      * @param val Date value.
      * @return This instance.
-     * @throws UnsupportedOperationException Not supported.
      */
     public ClientMessagePacker packDate(LocalDate val) {
         assert !closed : "Packer is closed";
 
-        throw new UnsupportedOperationException("TODO: IGNITE-15163");
+        byte[] data = new byte[6];
+
+        // TODO: Pack directly to ByteBuf without allocating IGNITE-15234.
+        ByteBuffer.wrap(data)
+            .putInt(val.getYear())
+            .put((byte)val.getMonthValue())
+            .put((byte)val.getDayOfMonth());
+
+        packExtensionTypeHeader(ClientMsgPackType.DATE, data.length);
+
+        addPayload(data);
+
+        return this;
     }
 
     /**
@@ -369,12 +411,24 @@ public class ClientMessagePacker extends MessagePacker {
      *
      * @param val Time value.
      * @return This instance.
-     * @throws UnsupportedOperationException Not supported.
      */
     public ClientMessagePacker packTime(LocalTime val) {
         assert !closed : "Packer is closed";
 
-        throw new UnsupportedOperationException("TODO: IGNITE-15163");
+        byte[] data = new byte[7];
+
+        // TODO: Pack directly to ByteBuf without allocating IGNITE-15234.
+        ByteBuffer.wrap(data)
+            .put((byte)val.getHour())
+            .put((byte)val.getMinute())
+            .put((byte)val.getSecond())
+            .putInt(val.getNano());
+
+        packExtensionTypeHeader(ClientMsgPackType.TIME, data.length);
+
+        addPayload(data);
+
+        return this;
     }
 
     /**
@@ -382,11 +436,25 @@ public class ClientMessagePacker extends MessagePacker {
      *
      * @param val Datetime value.
      * @return This instance.
-     * @throws UnsupportedOperationException Not supported.
      */
     public ClientMessagePacker packDateTime(LocalDateTime val) {
-        packDate(val.toLocalDate());
-        packTime(val.toLocalTime());
+        assert !closed : "Packer is closed";
+
+        byte[] data = new byte[13];
+
+        // TODO: Pack directly to ByteBuf without allocating IGNITE-15234.
+        ByteBuffer.wrap(data)
+            .putInt(val.getYear())
+            .put((byte)val.getMonthValue())
+            .put((byte)val.getDayOfMonth())
+            .put((byte)val.getHour())
+            .put((byte)val.getMinute())
+            .put((byte)val.getSecond())
+            .putInt(val.getNano());
+
+        packExtensionTypeHeader(ClientMsgPackType.DATETIME, data.length);
+
+        addPayload(data);
 
         return this;
     }
@@ -401,7 +469,18 @@ public class ClientMessagePacker extends MessagePacker {
     public ClientMessagePacker packTimestamp(Instant val) {
         assert !closed : "Packer is closed";
 
-        throw new UnsupportedOperationException("TODO: IGNITE-15163");
+        byte[] data = new byte[12];
+
+        // TODO: Pack directly to ByteBuf without allocating IGNITE-15234.
+        ByteBuffer.wrap(data)
+            .putLong(val.getEpochSecond())
+            .putInt(val.getNano());
+
+        packExtensionTypeHeader(ClientMsgPackType.TIMESTAMP, data.length);
+
+        addPayload(data);
+
+        return this;
     }
 
     /**
@@ -413,22 +492,34 @@ public class ClientMessagePacker extends MessagePacker {
      */
     public ClientMessagePacker packObject(Object val) {
         if (val == null)
-            return (ClientMessagePacker) packNil();
+            return (ClientMessagePacker)packNil();
+
+        if (val instanceof Byte)
+            return (ClientMessagePacker)packByte((byte)val);
+
+        if (val instanceof Short)
+            return (ClientMessagePacker)packShort((short)val);
 
         if (val instanceof Integer)
-            return (ClientMessagePacker) packInt((int) val);
+            return (ClientMessagePacker)packInt((int)val);
 
         if (val instanceof Long)
-            return (ClientMessagePacker) packLong((long) val);
+            return (ClientMessagePacker)packLong((long)val);
+
+        if (val instanceof Float)
+            return (ClientMessagePacker)packFloat((float)val);
+
+        if (val instanceof Double)
+            return (ClientMessagePacker)packDouble((double)val);
 
         if (val instanceof UUID)
-            return packUuid((UUID) val);
+            return packUuid((UUID)val);
 
         if (val instanceof String)
-            return (ClientMessagePacker) packString((String) val);
+            return (ClientMessagePacker)packString((String)val);
 
         if (val instanceof byte[]) {
-            byte[] bytes = (byte[]) val;
+            byte[] bytes = (byte[])val;
             packBinaryHeader(bytes.length);
             writePayload(bytes);
 
@@ -436,10 +527,13 @@ public class ClientMessagePacker extends MessagePacker {
         }
 
         if (val instanceof BigDecimal)
-            return packDecimal((BigDecimal) val);
+            return packDecimal((BigDecimal)val);
+
+        if (val instanceof BigInteger)
+            return packNumber((BigInteger)val);
 
         if (val instanceof BitSet)
-            return packBitSet((BitSet) val);
+            return packBitSet((BitSet)val);
 
         if (val instanceof LocalDate)
             return packDate((LocalDate)val);
@@ -453,7 +547,6 @@ public class ClientMessagePacker extends MessagePacker {
         if (val instanceof Instant)
             return packTimestamp((Instant)val);
 
-        // TODO: Support all basic types IGNITE-15163
         throw new UnsupportedOperationException("Unsupported type, can't serialize: " + val.getClass());
     }
 
diff --git a/modules/client-common/src/main/java/org/apache/ignite/client/proto/ClientMessageUnpacker.java b/modules/client-common/src/main/java/org/apache/ignite/client/proto/ClientMessageUnpacker.java
index cdf1e18..e812bc0 100644
--- a/modules/client-common/src/main/java/org/apache/ignite/client/proto/ClientMessageUnpacker.java
+++ b/modules/client-common/src/main/java/org/apache/ignite/client/proto/ClientMessageUnpacker.java
@@ -51,6 +51,7 @@ import static org.apache.ignite.client.proto.ClientDataType.INT16;
 import static org.apache.ignite.client.proto.ClientDataType.INT32;
 import static org.apache.ignite.client.proto.ClientDataType.INT64;
 import static org.apache.ignite.client.proto.ClientDataType.INT8;
+import static org.apache.ignite.client.proto.ClientDataType.NUMBER;
 import static org.apache.ignite.client.proto.ClientDataType.STRING;
 import static org.apache.ignite.client.proto.ClientDataType.TIME;
 import static org.apache.ignite.client.proto.ClientDataType.TIMESTAMP;
@@ -333,7 +334,7 @@ public class ClientMessageUnpacker extends MessageUnpacker {
         var len = hdr.getLength();
 
         if (type != ClientMsgPackType.UUID)
-            throw new MessageTypeException("Expected UUID extension (1), but got " + type);
+            throw new MessageTypeException("Expected UUID extension (3), but got " + type);
 
         if (len != 16)
             throw new MessageSizeException("Expected 16 bytes for UUID extension, but got " + len, len);
@@ -349,72 +350,170 @@ public class ClientMessageUnpacker extends MessageUnpacker {
      * Reads a decimal.
      *
      * @return Decimal value.
-     * @throws UnsupportedOperationException Not supported yet.
+     * @throws MessageTypeException when type is not Decimal.
      */
     public BigDecimal unpackDecimal() {
         assert refCnt > 0 : "Unpacker is closed";
 
-        throw new UnsupportedOperationException("TODO: IGNITE-15163");
+        var hdr = unpackExtensionTypeHeader();
+        var type = hdr.getType();
+        var len = hdr.getLength();
+
+        if (type != ClientMsgPackType.DECIMAL)
+            throw new MessageTypeException("Expected DECIMAL extension (2), but got " + type);
+
+        var bytes = readPayload(len);
+
+        ByteBuffer bb = ByteBuffer.wrap(bytes);
+
+        int scale = bb.getInt();
+
+        return new BigDecimal(new BigInteger(bytes, bb.position(), bb.remaining()), scale);
     }
 
     /**
      * Reads a bit set.
      *
      * @return Bit set.
-     * @throws UnsupportedOperationException Not supported yet.
+     * @throws MessageTypeException when type is not BitSet.
      */
     public BitSet unpackBitSet() {
         assert refCnt > 0 : "Unpacker is closed";
 
-        throw new UnsupportedOperationException("TODO: IGNITE-15163");
+        var hdr = unpackExtensionTypeHeader();
+        var type = hdr.getType();
+        var len = hdr.getLength();
+
+        if (type != ClientMsgPackType.BITMASK)
+            throw new MessageTypeException("Expected BITSET extension (7), but got " + type);
+
+        var bytes = readPayload(len);
+
+        return BitSet.valueOf(bytes);
+    }
+
+    /**
+     * Reads a number.
+     *
+     * @return BigInteger value.
+     * @throws MessageTypeException when type is not BigInteger.
+     */
+    public BigInteger unpackNumber() {
+        assert refCnt > 0 : "Unpacker is closed";
+
+        var hdr = unpackExtensionTypeHeader();
+        var type = hdr.getType();
+        var len = hdr.getLength();
+
+        if (type != ClientMsgPackType.NUMBER)
+            throw new MessageTypeException("Expected NUMBER extension (1), but got " + type);
+
+        var bytes = readPayload(len);
+
+        return new BigInteger(bytes);
     }
 
     /**
      * Reads a date.
      *
      * @return Date value.
-     * @throws UnsupportedOperationException Not supported yet.
+     * @throws MessageTypeException when type is not DATE.
+     * @throws MessageSizeException when size is not correct.
      */
     public LocalDate unpackDate() {
         assert refCnt > 0 : "Unpacker is closed";
 
-        throw new UnsupportedOperationException("TODO: IGNITE-15163");
+        var hdr = unpackExtensionTypeHeader();
+        var type = hdr.getType();
+        var len = hdr.getLength();
+
+        if (type != ClientMsgPackType.DATE)
+            throw new MessageTypeException("Expected DATE extension (4), but got " + type);
+
+        if (len != 6)
+            throw new MessageSizeException("Expected 6 bytes for DATE extension, but got " + len, len);
+
+        var data = ByteBuffer.wrap(readPayload(len));
+
+        return LocalDate.of(data.getInt(), data.get(), data.get());
     }
 
     /**
      * Reads a time.
      *
      * @return Time value.
-     * @throws UnsupportedOperationException Not supported yet.
+     * @throws MessageTypeException when type is not TIME.
+     * @throws MessageSizeException when size is not correct.
      */
     public LocalTime unpackTime() {
         assert refCnt > 0 : "Unpacker is closed";
 
-        throw new UnsupportedOperationException("TODO: IGNITE-15163");
+        var hdr = unpackExtensionTypeHeader();
+        var type = hdr.getType();
+        var len = hdr.getLength();
+
+        if (type != ClientMsgPackType.TIME)
+            throw new MessageTypeException("Expected TIME extension (5), but got " + type);
+
+        if (len != 7)
+            throw new MessageSizeException("Expected 7 bytes for TIME extension, but got " + len, len);
+
+        var data = ByteBuffer.wrap(readPayload(len));
+
+        return LocalTime.of(data.get(), data.get(), data.get(), data.getInt());
     }
 
     /**
      * Reads a datetime.
      *
      * @return Datetime value.
-     * @throws UnsupportedOperationException Not supported yet.
+     * @throws MessageTypeException when type is not DATETIME.
+     * @throws MessageSizeException when size is not correct.
      */
     public LocalDateTime unpackDateTime() {
         assert refCnt > 0 : "Unpacker is closed";
 
-        throw new UnsupportedOperationException("TODO: IGNITE-15163");
+        var hdr = unpackExtensionTypeHeader();
+        var type = hdr.getType();
+        var len = hdr.getLength();
+
+        if (type != ClientMsgPackType.DATETIME)
+            throw new MessageTypeException("Expected DATETIME extension (6), but got " + type);
+
+        if (len != 13)
+            throw new MessageSizeException("Expected 13 bytes for DATETIME extension, but got " + len, len);
+
+        var data = ByteBuffer.wrap(readPayload(len));
+
+        return LocalDateTime.of(
+            LocalDate.of(data.getInt(), data.get(), data.get()),
+            LocalTime.of(data.get(), data.get(), data.get(), data.getInt())
+        );
     }
 
     /**
      * Reads a timestamp.
      *
      * @return Timestamp value.
-     * @throws UnsupportedOperationException Not supported yet.
+     * @throws MessageTypeException when type is not TIMESTAMP.
+     * @throws MessageSizeException when size is not correct.
      */
     public Instant unpackTimestamp() {
         assert refCnt > 0 : "Unpacker is closed";
 
-        throw new UnsupportedOperationException("TODO: IGNITE-15163");
+        var hdr = unpackExtensionTypeHeader();
+        var type = hdr.getType();
+        var len = hdr.getLength();
+
+        if (type != ClientMsgPackType.TIMESTAMP)
+            throw new MessageTypeException("Expected TIMESTAMP extension (6), but got " + type);
+
+        if (len != 12)
+            throw new MessageSizeException("Expected 12 bytes for TIMESTAMP extension, but got " + len, len);
+
+        var data = ByteBuffer.wrap(readPayload(len));
+
+        return Instant.ofEpochSecond(data.getLong(), data.getInt());
     }
 
     /**
@@ -454,10 +553,11 @@ public class ClientMessageUnpacker extends MessageUnpacker {
             case STRING:
                 return unpackString();
 
-            case BYTES:
+            case BYTES: {
                 var cnt = unpackBinaryHeader();
 
                 return readPayload(cnt);
+            }
 
             case DECIMAL:
                 return unpackDecimal();
@@ -465,6 +565,9 @@ public class ClientMessageUnpacker extends MessageUnpacker {
             case BITMASK:
                 return unpackBitSet();
 
+            case NUMBER:
+                return unpackNumber();
+
             case DATE:
                 return unpackDate();
 
diff --git a/modules/client-common/src/main/java/org/apache/ignite/client/proto/ClientMsgPackType.java b/modules/client-common/src/main/java/org/apache/ignite/client/proto/ClientMsgPackType.java
index dd92c46..07fc7f0 100644
--- a/modules/client-common/src/main/java/org/apache/ignite/client/proto/ClientMsgPackType.java
+++ b/modules/client-common/src/main/java/org/apache/ignite/client/proto/ClientMsgPackType.java
@@ -38,4 +38,10 @@ public class ClientMsgPackType {
 
     /** DateTime. */
     public static final byte DATETIME = 6;
+
+    /** DateTime. */
+    public static final byte TIMESTAMP = 7;
+
+    /** Bit mask. */
+    public static final byte BITMASK = 8;
 }
diff --git a/modules/client-common/src/test/java/org/apache/ignite/client/proto/ClientMessagePackerUnpackerTest.java b/modules/client-common/src/test/java/org/apache/ignite/client/proto/ClientMessagePackerUnpackerTest.java
index 3d04434..613c190 100644
--- a/modules/client-common/src/test/java/org/apache/ignite/client/proto/ClientMessagePackerUnpackerTest.java
+++ b/modules/client-common/src/test/java/org/apache/ignite/client/proto/ClientMessagePackerUnpackerTest.java
@@ -18,17 +18,31 @@
 package org.apache.ignite.client.proto;
 
 import java.io.IOException;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.BitSet;
+import java.util.Random;
 import java.util.UUID;
 import io.netty.buffer.PooledByteBufAllocator;
 import io.netty.buffer.Unpooled;
+import org.apache.ignite.internal.testframework.IgniteTestUtils;
 import org.junit.jupiter.api.Test;
 
+import static org.apache.ignite.internal.testframework.IgniteTestUtils.randomBytes;
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
 /**
  * Tests Ignite-specific MsgPack extensions.
  */
 public class ClientMessagePackerUnpackerTest {
+    /** Random. */
+    private final Random rnd = new Random();
+
     @Test
     public void testPackerCloseReleasesPooledBuffer() {
         var buf = PooledByteBufAllocator.DEFAULT.directBuffer();
@@ -73,6 +87,120 @@ public class ClientMessagePackerUnpackerTest {
         testUUID(new UUID(0, 0));
     }
 
+    @Test
+    public void testNumber() throws IOException {
+        testNumber(BigInteger.ZERO);
+        testNumber(BigInteger.valueOf(Long.MIN_VALUE));
+        testNumber(BigInteger.valueOf(Long.MAX_VALUE));
+
+        testNumber(new BigInteger(randomBytes(rnd, 100)));
+        testNumber(new BigInteger(randomBytes(rnd, 250)));
+        testNumber(new BigInteger(randomBytes(rnd, 1000)));
+    }
+
+    @Test
+    public void testDecimal() throws IOException {
+        testDecimal(BigDecimal.ZERO);
+        testDecimal(BigDecimal.valueOf(Long.MIN_VALUE));
+        testDecimal(BigDecimal.valueOf(Long.MAX_VALUE));
+
+        testDecimal(new BigDecimal(new BigInteger(randomBytes(rnd, 100)), 50));
+        testDecimal(new BigDecimal(new BigInteger(randomBytes(rnd, 250)), 200));
+        testDecimal(new BigDecimal(new BigInteger(randomBytes(rnd, 1000)), 500));
+    }
+
+    @Test
+    public void testBitSet() throws IOException {
+        testBitSet(BitSet.valueOf(new byte[0]));
+        testBitSet(BitSet.valueOf(randomBytes(rnd, 1)));
+        testBitSet(BitSet.valueOf(randomBytes(rnd, 100)));
+        testBitSet(BitSet.valueOf(randomBytes(rnd, 1000)));
+    }
+
+    @Test
+    public void testTemporalTypes() throws IOException {
+        try (var packer = new ClientMessagePacker(PooledByteBufAllocator.DEFAULT.directBuffer())) {
+            LocalDate date = LocalDate.now();
+            LocalTime time = LocalTime.now();
+            Instant timestamp = Instant.now();
+
+            packer.packDate(date);
+            packer.packTime(time);
+            packer.packTimestamp(timestamp);
+            packer.packDateTime(LocalDateTime.of(date, time));
+
+            var buf = packer.getBuffer();
+            //noinspection unused
+            var len = buf.readInt();
+
+            byte[] data = new byte[buf.readableBytes()];
+            buf.readBytes(data);
+
+            try (var unpacker = new ClientMessageUnpacker(Unpooled.wrappedBuffer(data))) {
+                assertEquals(date, unpacker.unpackDate());
+                assertEquals(time, unpacker.unpackTime());
+                assertEquals(timestamp, unpacker.unpackTimestamp());
+                assertEquals(LocalDateTime.of(date, time), unpacker.unpackDateTime());
+            }
+        }
+    }
+
+    @Test
+    public void testVariousTypesSupport() throws IOException {
+        Object[] values = new Object[]{
+            (byte)1, (short)2, 3, 4L, 5.5f, 6.6d,
+            BigDecimal.valueOf(rnd.nextLong()),
+            UUID.randomUUID(),
+            IgniteTestUtils.randomString(rnd, 11),
+            IgniteTestUtils.randomBytes(rnd, 22),
+            IgniteTestUtils.randomBitSet(rnd, 33),
+            LocalDate.now(),
+            LocalTime.now(),
+            LocalDateTime.now(),
+            Instant.now()
+        };
+
+        try (var packer = new ClientMessagePacker(PooledByteBufAllocator.DEFAULT.directBuffer())) {
+            for (Object val : values)
+                packer.packObject(val);
+
+            var buf = packer.getBuffer();
+            //noinspection unused
+            var len = buf.readInt();
+
+            byte[] data = new byte[buf.readableBytes()];
+            buf.readBytes(data);
+
+            try (var unpacker = new ClientMessageUnpacker(Unpooled.wrappedBuffer(data))) {
+                for (int i = 0; i < values.length; i++) {
+                    if (values[i] instanceof byte[])
+                        assertArrayEquals((byte[])values[i], (byte[])unpacker.unpackObject(i + 1));
+                    else
+                        assertEquals(values[i], unpacker.unpackObject(i + 1));
+                }
+            }
+        }
+    }
+
+    private void testBitSet(BitSet val) {
+        try (var packer = new ClientMessagePacker(PooledByteBufAllocator.DEFAULT.directBuffer())) {
+            packer.packBitSet(val);
+
+            var buf = packer.getBuffer();
+            //noinspection unused
+            var len = buf.readInt();
+
+            byte[] data = new byte[buf.readableBytes()];
+            buf.readBytes(data);
+
+            try (var unpacker = new ClientMessageUnpacker(Unpooled.wrappedBuffer(data))) {
+                var res = unpacker.unpackBitSet();
+
+                assertEquals(val, res);
+            }
+        }
+    }
+
     private void testUUID(UUID u) throws IOException {
         try (var packer = new ClientMessagePacker(PooledByteBufAllocator.DEFAULT.directBuffer())) {
             packer.packUuid(u);
@@ -91,4 +219,42 @@ public class ClientMessagePackerUnpackerTest {
             }
         }
     }
+
+    private void testNumber(BigInteger val) {
+        try (var packer = new ClientMessagePacker(PooledByteBufAllocator.DEFAULT.directBuffer())) {
+            packer.packNumber(val);
+
+            var buf = packer.getBuffer();
+            //noinspection unused
+            var len = buf.readInt();
+
+            byte[] data = new byte[buf.readableBytes()];
+            buf.readBytes(data);
+
+            try (var unpacker = new ClientMessageUnpacker(Unpooled.wrappedBuffer(data))) {
+                var res = unpacker.unpackNumber();
+
+                assertEquals(val, res);
+            }
+        }
+    }
+
+    private void testDecimal(BigDecimal val) {
+        try (var packer = new ClientMessagePacker(PooledByteBufAllocator.DEFAULT.directBuffer())) {
+            packer.packDecimal(val);
+
+            var buf = packer.getBuffer();
+            //noinspection unused
+            var len = buf.readInt();
+
+            byte[] data = new byte[buf.readableBytes()];
+            buf.readBytes(data);
+
+            try (var unpacker = new ClientMessageUnpacker(Unpooled.wrappedBuffer(data))) {
+                var res = unpacker.unpackDecimal();
+
+                assertEquals(val, res);
+            }
+        }
+    }
 }
diff --git a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTableCommon.java b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTableCommon.java
index 15cfa33..93ef9e9 100644
--- a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTableCommon.java
+++ b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/table/ClientTableCommon.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.client.handler.requests.table;
 
 import java.math.BigDecimal;
+import java.math.BigInteger;
 import java.time.Instant;
 import java.time.LocalDate;
 import java.time.LocalDateTime;
@@ -360,6 +361,9 @@ class ClientTableCommon {
             case DECIMAL:
                 return ClientDataType.DECIMAL;
 
+            case NUMBER:
+                return ClientDataType.NUMBER;
+
             case UUID:
                 return ClientDataType.UUID;
 
@@ -425,6 +429,10 @@ class ClientTableCommon {
                 packer.packDecimal((BigDecimal)val);
                 break;
 
+            case NUMBER:
+                packer.packNumber((BigInteger)val);
+                break;
+
             case UUID:
                 packer.packUuid((UUID)val);
                 break;
diff --git a/modules/schema/pom.xml b/modules/schema/pom.xml
index 783b5f0..6faa0d5 100644
--- a/modules/schema/pom.xml
+++ b/modules/schema/pom.xml
@@ -62,14 +62,14 @@
         <!-- Test dependencies -->
         <dependency>
             <groupId>org.apache.ignite</groupId>
-            <artifactId>ignite-core</artifactId>
+            <artifactId>ignite-configuration</artifactId>
             <scope>test</scope>
             <type>test-jar</type>
         </dependency>
 
         <dependency>
             <groupId>org.apache.ignite</groupId>
-            <artifactId>ignite-configuration</artifactId>
+            <artifactId>ignite-core</artifactId>
             <scope>test</scope>
             <type>test-jar</type>
         </dependency>
diff --git a/modules/schema/src/test/java/org/apache/ignite/internal/schema/TestUtils.java b/modules/schema/src/test/java/org/apache/ignite/internal/schema/TestUtils.java
index 9b307e7..afda470 100644
--- a/modules/schema/src/test/java/org/apache/ignite/internal/schema/TestUtils.java
+++ b/modules/schema/src/test/java/org/apache/ignite/internal/schema/TestUtils.java
@@ -26,13 +26,11 @@ import java.time.LocalTime;
 import java.time.Year;
 import java.time.temporal.ChronoUnit;
 import java.util.Random;
+import org.apache.ignite.internal.testframework.IgniteTestUtils;
 
 import static org.apache.ignite.internal.schema.row.TemporalTypesHelper.MAX_YEAR;
 import static org.apache.ignite.internal.schema.row.TemporalTypesHelper.MIN_YEAR;
 import static org.apache.ignite.internal.schema.row.TemporalTypesHelper.normalizeNanos;
-import static org.apache.ignite.internal.testframework.IgniteTestUtils.randomBitSet;
-import static org.apache.ignite.internal.testframework.IgniteTestUtils.randomBytes;
-import static org.apache.ignite.internal.testframework.IgniteTestUtils.randomString;
 
 /**
  * Test utility class.
@@ -69,10 +67,10 @@ public final class TestUtils {
                 return new java.util.UUID(rnd.nextLong(), rnd.nextLong());
 
             case STRING:
-                return randomString(rnd, rnd.nextInt(255));
+                return IgniteTestUtils.randomString(rnd, rnd.nextInt(255));
 
             case BYTES:
-                return randomBytes(rnd, rnd.nextInt(255));
+                return IgniteTestUtils.randomBytes(rnd, rnd.nextInt(255));
 
             case NUMBER:
                 return BigInteger.probablePrime(12, rnd);
@@ -83,7 +81,7 @@ public final class TestUtils {
             case BITMASK: {
                 BitmaskNativeType maskType = (BitmaskNativeType)type;
 
-                return randomBitSet(rnd, maskType.bits());
+                return IgniteTestUtils.randomBitSet(rnd, maskType.bits());
             }
 
             case DATE: {