You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by is...@apache.org on 2018/10/15 10:29:53 UTC

[2/6] ignite git commit: IGNITE-7782 Python thin client

http://git-wip-us.apache.org/repos/asf/ignite/blob/7e547b13/modules/platforms/python/pyignite/datatypes/complex.py
----------------------------------------------------------------------
diff --git a/modules/platforms/python/pyignite/datatypes/complex.py b/modules/platforms/python/pyignite/datatypes/complex.py
new file mode 100644
index 0000000..9a5664c
--- /dev/null
+++ b/modules/platforms/python/pyignite/datatypes/complex.py
@@ -0,0 +1,531 @@
+# 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.
+
+from collections import OrderedDict
+import ctypes
+import inspect
+
+from pyignite.constants import *
+from pyignite.exceptions import ParseError
+from pyignite.utils import entity_id, hashcode, is_hinted
+from .internal import AnyDataObject
+from .type_codes import *
+
+
+__all__ = [
+    'Map', 'ObjectArrayObject', 'CollectionObject', 'MapObject',
+    'WrappedDataObject', 'BinaryObject',
+]
+
+
+class ObjectArrayObject:
+    """
+    Array of objects of any type. Its Python representation is
+    tuple(type_id, iterable of any type).
+    """
+    type_code = TC_OBJECT_ARRAY
+    type_or_id_name = 'type_id'
+
+    @classmethod
+    def build_header(cls):
+        return type(
+            cls.__name__+'Header',
+            (ctypes.LittleEndianStructure,),
+            {
+                '_pack_': 1,
+                '_fields_': [
+                    ('type_code', ctypes.c_byte),
+                    ('type_id', ctypes.c_int),
+                    ('length', ctypes.c_int),
+                ],
+            }
+        )
+
+    @classmethod
+    def parse(cls, client: 'Client'):
+        header_class = cls.build_header()
+        buffer = client.recv(ctypes.sizeof(header_class))
+        header = header_class.from_buffer_copy(buffer)
+        fields = []
+
+        for i in range(header.length):
+            c_type, buffer_fragment = AnyDataObject.parse(client)
+            buffer += buffer_fragment
+            fields.append(('element_{}'.format(i), c_type))
+
+        final_class = type(
+            cls.__name__,
+            (header_class,),
+            {
+                '_pack_': 1,
+                '_fields_': fields,
+            }
+        )
+        return final_class, buffer
+
+    @classmethod
+    def to_python(cls, ctype_object, *args, **kwargs):
+        result = []
+        for i in range(ctype_object.length):
+            result.append(
+                AnyDataObject.to_python(
+                    getattr(ctype_object, 'element_{}'.format(i)),
+                    *args, **kwargs
+                )
+            )
+        return getattr(ctype_object, cls.type_or_id_name), result
+
+    @classmethod
+    def from_python(cls, value):
+        type_or_id, value = value
+        header_class = cls.build_header()
+        header = header_class()
+        header.type_code = int.from_bytes(
+            cls.type_code,
+            byteorder=PROTOCOL_BYTE_ORDER
+        )
+        try:
+            length = len(value)
+        except TypeError:
+            value = [value]
+            length = 1
+        header.length = length
+        setattr(header, cls.type_or_id_name, type_or_id)
+        buffer = bytes(header)
+
+        for x in value:
+            buffer += AnyDataObject.from_python(x)
+        return buffer
+
+
+class WrappedDataObject:
+    """
+    One or more binary objects can be wrapped in an array. This allows reading,
+    storing, passing and writing objects efficiently without understanding
+    their contents, performing simple byte copy.
+
+    Python representation: tuple(payload: bytes, offset: integer). Offset
+    points to the root object of the array.
+    """
+    type_code = TC_ARRAY_WRAPPED_OBJECTS
+
+    @classmethod
+    def build_header(cls):
+        return type(
+            cls.__name__+'Header',
+            (ctypes.LittleEndianStructure,),
+            {
+                '_pack_': 1,
+                '_fields_': [
+                    ('type_code', ctypes.c_byte),
+                    ('length', ctypes.c_int),
+                ],
+            }
+        )
+
+    @classmethod
+    def parse(cls, client: 'Client'):
+        header_class = cls.build_header()
+        buffer = client.recv(ctypes.sizeof(header_class))
+        header = header_class.from_buffer_copy(buffer)
+
+        final_class = type(
+            cls.__name__,
+            (header_class,),
+            {
+                '_pack_': 1,
+                '_fields_': [
+                    ('payload', ctypes.c_byte*header.length),
+                    ('offset', ctypes.c_int),
+                ],
+            }
+        )
+        buffer += client.recv(
+            ctypes.sizeof(final_class) - ctypes.sizeof(header_class)
+        )
+        return final_class, buffer
+
+    @classmethod
+    def to_python(cls, ctype_object, *args, **kwargs):
+        return bytes(ctype_object.payload), ctype_object.offset
+
+    @classmethod
+    def from_python(cls, value):
+        raise ParseError('Send unwrapped data.')
+
+
+class CollectionObject(ObjectArrayObject):
+    """
+    Just like object array, but contains deserialization type hint instead of
+    type id. This hint is also useless in Python, because the list type along
+    covers all the use cases.
+
+    Also represented as tuple(type_id, iterable of any type) in Python.
+    """
+    type_code = TC_COLLECTION
+    type_or_id_name = 'type'
+    pythonic = list
+    default = []
+
+    @classmethod
+    def build_header(cls):
+        return type(
+            cls.__name__+'Header',
+            (ctypes.LittleEndianStructure,),
+            {
+                '_pack_': 1,
+                '_fields_': [
+                    ('type_code', ctypes.c_byte),
+                    ('length', ctypes.c_int),
+                    ('type', ctypes.c_byte),
+                ],
+            }
+        )
+
+
+class Map:
+    """
+    Dictionary type, payload-only.
+
+    Ignite does not track the order of key-value pairs in its caches, hence
+    the ordinary Python dict type, not the collections.OrderedDict.
+    """
+    HASH_MAP = 1
+    LINKED_HASH_MAP = 2
+
+    @classmethod
+    def build_header(cls):
+        return type(
+            cls.__name__+'Header',
+            (ctypes.LittleEndianStructure,),
+            {
+                '_pack_': 1,
+                '_fields_': [
+                    ('length', ctypes.c_int),
+                ],
+            }
+        )
+
+    @classmethod
+    def parse(cls, client: 'Client'):
+        header_class = cls.build_header()
+        buffer = client.recv(ctypes.sizeof(header_class))
+        header = header_class.from_buffer_copy(buffer)
+        fields = []
+
+        for i in range(header.length << 1):
+            c_type, buffer_fragment = AnyDataObject.parse(client)
+            buffer += buffer_fragment
+            fields.append(('element_{}'.format(i), c_type))
+
+        final_class = type(
+            cls.__name__,
+            (header_class,),
+            {
+                '_pack_': 1,
+                '_fields_': fields,
+            }
+        )
+        return final_class, buffer
+
+    @classmethod
+    def to_python(cls, ctype_object, *args, **kwargs):
+        map_type = getattr(ctype_object, 'type', cls.HASH_MAP)
+        result = OrderedDict() if map_type == cls.LINKED_HASH_MAP else {}
+
+        for i in range(0, ctype_object.length << 1, 2):
+            k = AnyDataObject.to_python(
+                    getattr(ctype_object, 'element_{}'.format(i)),
+                    *args, **kwargs
+                )
+            v = AnyDataObject.to_python(
+                    getattr(ctype_object, 'element_{}'.format(i + 1)),
+                    *args, **kwargs
+                )
+            result[k] = v
+        return result
+
+    @classmethod
+    def from_python(cls, value, type_id=None):
+        header_class = cls.build_header()
+        header = header_class()
+        length = len(value)
+        header.length = length
+        if hasattr(header, 'type_code'):
+            header.type_code = int.from_bytes(
+                cls.type_code,
+                byteorder=PROTOCOL_BYTE_ORDER
+            )
+        if hasattr(header, 'type'):
+            header.type = type_id
+        buffer = bytes(header)
+
+        for k, v in value.items():
+            if is_hinted(k):
+                buffer += k[1].from_python(k[0])
+            else:
+                buffer += AnyDataObject.from_python(k)
+            if is_hinted(v):
+                buffer += v[1].from_python(v[0])
+            else:
+                buffer += AnyDataObject.from_python(v)
+        return buffer
+
+
+class MapObject(Map):
+    """
+    This is a dictionary type. Type conversion hint can be a `HASH_MAP`
+    (ordinary dict) or `LINKED_HASH_MAP` (collections.OrderedDict).
+
+    Keys and values in map are independent data objects, but `count`
+    counts pairs. Very annoying.
+    """
+    type_code = TC_MAP
+    pythonic = dict
+    default = {}
+
+    @classmethod
+    def build_header(cls):
+        return type(
+            cls.__name__+'Header',
+            (ctypes.LittleEndianStructure,),
+            {
+                '_pack_': 1,
+                '_fields_': [
+                    ('type_code', ctypes.c_byte),
+                    ('length', ctypes.c_int),
+                    ('type', ctypes.c_byte),
+                ],
+            }
+        )
+
+    @classmethod
+    def to_python(cls, ctype_object, *args, **kwargs):
+        return ctype_object.type, super().to_python(
+            ctype_object, *args, **kwargs
+        )
+
+    @classmethod
+    def from_python(cls, value):
+        type_id, value = value
+        return super().from_python(value, type_id)
+
+
+class BinaryObject:
+    type_code = TC_COMPLEX_OBJECT
+
+    USER_TYPE = 0x0001
+    HAS_SCHEMA = 0x0002
+    HAS_RAW_DATA = 0x0004
+    OFFSET_ONE_BYTE = 0x0008
+    OFFSET_TWO_BYTES = 0x0010
+    COMPACT_FOOTER = 0x0020
+
+    @classmethod
+    def build_header(cls):
+        return type(
+            cls.__name__,
+            (ctypes.LittleEndianStructure,),
+            {
+                '_pack_': 1,
+                '_fields_': [
+                    ('type_code', ctypes.c_byte),
+                    ('version', ctypes.c_byte),
+                    ('flags', ctypes.c_short),
+                    ('type_id', ctypes.c_int),
+                    ('hash_code', ctypes.c_int),
+                    ('length', ctypes.c_int),
+                    ('schema_id', ctypes.c_int),
+                    ('schema_offset', ctypes.c_int),
+                ],
+            }
+        )
+
+    @classmethod
+    def offset_c_type(cls, flags: int):
+        if flags & cls.OFFSET_ONE_BYTE:
+            return ctypes.c_ubyte
+        if flags & cls.OFFSET_TWO_BYTES:
+            return ctypes.c_uint16
+        return ctypes.c_uint
+
+    @classmethod
+    def schema_type(cls, flags: int):
+        if flags & cls.COMPACT_FOOTER:
+            return cls.offset_c_type(flags)
+        return type(
+            'SchemaElement',
+            (ctypes.LittleEndianStructure,),
+            {
+                '_pack_': 1,
+                '_fields_': [
+                    ('field_id', ctypes.c_int),
+                    ('offset', cls.offset_c_type(flags)),
+                ],
+            },
+        )
+
+    @staticmethod
+    def get_dataclass(client: 'Client', header) -> OrderedDict:
+        # get field names from outer space
+        temp_conn = client.clone()
+        result = temp_conn.query_binary_type(header.type_id, header.schema_id)
+        temp_conn.close()
+        if not result:
+            raise ParseError('Binary type is not registered')
+        return result
+
+    @classmethod
+    def parse(cls, client: 'Client'):
+        from pyignite.datatypes import Struct
+
+        header_class = cls.build_header()
+        buffer = client.recv(ctypes.sizeof(header_class))
+        header = header_class.from_buffer_copy(buffer)
+
+        # ignore full schema, always retrieve fields' types and order
+        # from complex types registry
+        data_class = cls.get_dataclass(client, header)
+        fields = data_class.schema.items()
+        object_fields_struct = Struct(fields)
+        object_fields, object_fields_buffer = object_fields_struct.parse(client)
+        buffer += object_fields_buffer
+        final_class_fields = [('object_fields', object_fields)]
+
+        if header.flags & cls.HAS_SCHEMA:
+            schema = cls.schema_type(header.flags) * len(fields)
+            buffer += client.recv(ctypes.sizeof(schema))
+            final_class_fields.append(('schema', schema))
+
+        final_class = type(
+            cls.__name__,
+            (header_class,),
+            {
+                '_pack_': 1,
+                '_fields_': final_class_fields,
+            }
+        )
+        # register schema encoding approach
+        client.compact_footer = bool(header.flags & cls.COMPACT_FOOTER)
+        return final_class, buffer
+
+    @classmethod
+    def to_python(cls, ctype_object, client: 'Client'=None, *args, **kwargs):
+
+        if not client:
+            raise ParseError(
+                'Can not query binary type {}'.format(ctype_object.type_id)
+            )
+
+        data_class = client.query_binary_type(
+            ctype_object.type_id,
+            ctype_object.schema_id
+        )
+        result = data_class()
+
+        result.version = ctype_object.version
+        for field_name, field_type in data_class.schema.items():
+            setattr(
+                result, field_name, field_type.to_python(
+                    getattr(ctype_object.object_fields, field_name),
+                    client, *args, **kwargs
+                )
+            )
+        return result
+
+    @classmethod
+    def from_python(cls, value: object):
+
+        def find_client():
+            """
+            A nice hack. Extracts the nearest `Client` instance from the
+            call stack.
+            """
+            from pyignite import Client
+
+            frame = None
+            try:
+                for rec in inspect.stack()[2:]:
+                    frame = rec[0]
+                    code = frame.f_code
+                    for varname in code.co_varnames:
+                        suspect = frame.f_locals[varname]
+                        if isinstance(suspect, Client):
+                            return suspect
+            finally:
+                del frame
+
+        compact_footer = True  # this is actually used
+        client = find_client()
+        if client:
+            # if no client can be found, the class of the `value` is discarded
+            # and the new dataclass is automatically registered later on
+            client.register_binary_type(value.__class__)
+            compact_footer = client.compact_footer
+        else:
+            raise Warning(
+                'Can not register binary type {}'.format(value.type_name)
+            )
+
+        # prepare header
+        header_class = cls.build_header()
+        header = header_class()
+        header.type_code = int.from_bytes(
+            cls.type_code,
+            byteorder=PROTOCOL_BYTE_ORDER
+        )
+
+        header.flags = cls.USER_TYPE | cls.HAS_SCHEMA
+        if compact_footer:
+            header.flags |= cls.COMPACT_FOOTER
+        header.version = value.version
+        header.type_id = value.type_id
+        header.schema_id = value.schema_id
+
+        # create fields and calculate offsets
+        field_buffer = b''
+        offsets = [ctypes.sizeof(header_class)]
+        schema_items = list(value.schema.items())
+        for field_name, field_type in schema_items:
+            partial_buffer = field_type.from_python(
+                getattr(
+                    value, field_name, getattr(field_type, 'default', None)
+                )
+            )
+            offsets.append(max(offsets) + len(partial_buffer))
+            field_buffer += partial_buffer
+
+        offsets = offsets[:-1]
+
+        # create footer
+        if max(offsets, default=0) < 255:
+            header.flags |= cls.OFFSET_ONE_BYTE
+        elif max(offsets) < 65535:
+            header.flags |= cls.OFFSET_TWO_BYTES
+        schema_class = cls.schema_type(header.flags) * len(offsets)
+        schema = schema_class()
+        if compact_footer:
+            for i, offset in enumerate(offsets):
+                schema[i] = offset
+        else:
+            for i, offset in enumerate(offsets):
+                schema[i].field_id = entity_id(schema_items[i][0])
+                schema[i].offset = offset
+        # calculate size and hash code
+        header.schema_offset = ctypes.sizeof(header_class) + len(field_buffer)
+        header.length = header.schema_offset + ctypes.sizeof(schema_class)
+        header.hash_code = hashcode(field_buffer + bytes(schema))
+
+        return bytes(header) + field_buffer + bytes(schema)

http://git-wip-us.apache.org/repos/asf/ignite/blob/7e547b13/modules/platforms/python/pyignite/datatypes/internal.py
----------------------------------------------------------------------
diff --git a/modules/platforms/python/pyignite/datatypes/internal.py b/modules/platforms/python/pyignite/datatypes/internal.py
new file mode 100644
index 0000000..a363a5f
--- /dev/null
+++ b/modules/platforms/python/pyignite/datatypes/internal.py
@@ -0,0 +1,461 @@
+# 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.
+
+from collections import OrderedDict
+import ctypes
+import decimal
+from datetime import date, datetime, timedelta
+from typing import Any, Tuple
+import uuid
+
+import attr
+
+from pyignite.constants import *
+from pyignite.exceptions import ParseError
+from pyignite.utils import is_binary, is_hinted, is_iterable
+from .type_codes import *
+
+
+__all__ = ['AnyDataArray', 'AnyDataObject', 'Struct', 'StructArray', 'tc_map']
+
+
+def tc_map(key: bytes, _memo_map: dict={}):
+    """
+    Returns a default parser/generator class for the given type code.
+
+    This mapping is used internally inside listed complex parser/generator
+    classes, so it has to be a function. Local imports are used for the same
+    reason.
+
+    :param key: Ignite type code,
+    :param _memo_map: do not use this parameter, it is for memoization
+     of the “type code-type class” mapping,
+    :return: parser/generator class for the type code.
+    """
+    if not _memo_map:
+        from pyignite.datatypes import (
+            Null, ByteObject, ShortObject, IntObject, LongObject, FloatObject,
+            DoubleObject, CharObject, BoolObject, UUIDObject, DateObject,
+            TimestampObject, TimeObject, EnumObject, BinaryEnumObject,
+            ByteArrayObject, ShortArrayObject, IntArrayObject, LongArrayObject,
+            FloatArrayObject, DoubleArrayObject, CharArrayObject,
+            BoolArrayObject,
+            UUIDArrayObject, DateArrayObject, TimestampArrayObject,
+            TimeArrayObject, EnumArrayObject, String, StringArrayObject,
+            DecimalObject, DecimalArrayObject, ObjectArrayObject,
+            CollectionObject,
+            MapObject, BinaryObject, WrappedDataObject,
+        )
+
+        _memo_map = {
+            TC_NULL: Null,
+
+            TC_BYTE: ByteObject,
+            TC_SHORT: ShortObject,
+            TC_INT: IntObject,
+            TC_LONG: LongObject,
+            TC_FLOAT: FloatObject,
+            TC_DOUBLE: DoubleObject,
+            TC_CHAR: CharObject,
+            TC_BOOL: BoolObject,
+
+            TC_UUID: UUIDObject,
+            TC_DATE: DateObject,
+            TC_TIMESTAMP: TimestampObject,
+            TC_TIME: TimeObject,
+            TC_ENUM: EnumObject,
+            TC_BINARY_ENUM: BinaryEnumObject,
+
+            TC_BYTE_ARRAY: ByteArrayObject,
+            TC_SHORT_ARRAY: ShortArrayObject,
+            TC_INT_ARRAY: IntArrayObject,
+            TC_LONG_ARRAY: LongArrayObject,
+            TC_FLOAT_ARRAY: FloatArrayObject,
+            TC_DOUBLE_ARRAY: DoubleArrayObject,
+            TC_CHAR_ARRAY: CharArrayObject,
+            TC_BOOL_ARRAY: BoolArrayObject,
+
+            TC_UUID_ARRAY: UUIDArrayObject,
+            TC_DATE_ARRAY: DateArrayObject,
+            TC_TIMESTAMP_ARRAY: TimestampArrayObject,
+            TC_TIME_ARRAY: TimeArrayObject,
+            TC_ENUM_ARRAY: EnumArrayObject,
+
+            TC_STRING: String,
+            TC_STRING_ARRAY: StringArrayObject,
+            TC_DECIMAL: DecimalObject,
+            TC_DECIMAL_ARRAY: DecimalArrayObject,
+
+            TC_OBJECT_ARRAY: ObjectArrayObject,
+            TC_COLLECTION: CollectionObject,
+            TC_MAP: MapObject,
+
+            TC_COMPLEX_OBJECT: BinaryObject,
+            TC_ARRAY_WRAPPED_OBJECTS: WrappedDataObject,
+        }
+    return _memo_map[key]
+
+
+@attr.s
+class StructArray:
+    """ `counter_type` counter, followed by count*following structure. """
+    following = attr.ib(type=list, factory=list)
+    counter_type = attr.ib(default=ctypes.c_int)
+    defaults = attr.ib(type=dict, default={})
+
+    def build_header_class(self):
+        return type(
+            self.__class__.__name__+'Header',
+            (ctypes.LittleEndianStructure,),
+            {
+                '_pack_': 1,
+                '_fields_': [
+                    ('length', self.counter_type),
+                ],
+            },
+        )
+
+    def parse(self, client: 'Client'):
+        buffer = client.recv(ctypes.sizeof(self.counter_type))
+        length = int.from_bytes(buffer, byteorder=PROTOCOL_BYTE_ORDER)
+        fields = []
+
+        for i in range(length):
+            c_type, buffer_fragment = Struct(self.following).parse(client)
+            buffer += buffer_fragment
+            fields.append(('element_{}'.format(i), c_type))
+
+        data_class = type(
+            'StructArray',
+            (self.build_header_class(),),
+            {
+                '_pack_': 1,
+                '_fields_': fields,
+            },
+        )
+
+        return data_class, buffer
+
+    def to_python(self, ctype_object, *args, **kwargs):
+        result = []
+        length = getattr(ctype_object, 'length', 0)
+        for i in range(length):
+            result.append(
+                Struct(
+                    self.following, dict_type=dict
+                ).to_python(
+                    getattr(ctype_object, 'element_{}'.format(i)),
+                    *args, **kwargs
+                )
+            )
+        return result
+
+    def from_python(self, value):
+        length = len(value)
+        header_class = self.build_header_class()
+        header = header_class()
+        header.length = length
+        buffer = bytes(header)
+
+        for i, v in enumerate(value):
+            for default_key, default_value in self.defaults.items():
+                v.setdefault(default_key, default_value)
+            for name, el_class in self.following:
+                buffer += el_class.from_python(v[name])
+
+        return buffer
+
+
+@attr.s
+class Struct:
+    """ Sequence of fields, including variable-sized and nested. """
+    fields = attr.ib(type=list)
+    dict_type = attr.ib(default=OrderedDict)
+    defaults = attr.ib(type=dict, default={})
+
+    def parse(self, client: 'Client') -> Tuple[type, bytes]:
+        buffer = b''
+        fields = []
+
+        for name, c_type in self.fields:
+            c_type, buffer_fragment = c_type.parse(client)
+            buffer += buffer_fragment
+
+            fields.append((name, c_type))
+
+        data_class = type(
+            'Struct',
+            (ctypes.LittleEndianStructure,),
+            {
+                '_pack_': 1,
+                '_fields_': fields,
+            },
+        )
+
+        return data_class, buffer
+
+    def to_python(self, ctype_object, *args, **kwargs) -> Any:
+        result = self.dict_type()
+        for name, c_type in self.fields:
+            result[name] = c_type.to_python(
+                getattr(ctype_object, name),
+                *args, **kwargs
+            )
+        return result
+
+    def from_python(self, value) -> bytes:
+        buffer = b''
+
+        for default_key, default_value in self.defaults.items():
+            value.setdefault(default_key, default_value)
+
+        for name, el_class in self.fields:
+            buffer += el_class.from_python(value[name])
+
+        return buffer
+
+
+class AnyDataObject:
+    """
+    Not an actual Ignite type, but contains a guesswork
+    on serializing Python data or parsing an unknown Ignite data object.
+    """
+    _python_map = None
+    _python_array_map = None
+
+    @staticmethod
+    def get_subtype(iterable, allow_none=False):
+        # arrays of these types can contain Null objects
+        object_array_python_types = [
+            str,
+            datetime,
+            timedelta,
+            decimal.Decimal,
+            uuid.UUID,
+        ]
+
+        iterator = iter(iterable)
+        type_first = type(None)
+        try:
+            while isinstance(None, type_first):
+                type_first = type(next(iterator))
+        except StopIteration:
+            raise TypeError(
+                'Can not represent an empty iterable '
+                'or an iterable of `NoneType` in Ignite type.'
+            )
+
+        if type_first in object_array_python_types:
+            allow_none = True
+
+        # if an iterable contains items of more than one non-nullable type,
+        # return None
+        if all([
+            isinstance(x, type_first)
+            or ((x is None) and allow_none) for x in iterator
+        ]):
+            return type_first
+
+    @classmethod
+    def parse(cls, client: 'Client'):
+        type_code = client.recv(ctypes.sizeof(ctypes.c_byte))
+        try:
+            data_class = tc_map(type_code)
+        except KeyError:
+            raise ParseError('Unknown type code: `{}`'.format(type_code))
+        client.prefetch += type_code
+        return data_class.parse(client)
+
+    @classmethod
+    def to_python(cls, ctype_object, *args, **kwargs):
+        type_code = ctype_object.type_code.to_bytes(
+            ctypes.sizeof(ctypes.c_byte),
+            byteorder=PROTOCOL_BYTE_ORDER
+        )
+        data_class = tc_map(type_code)
+        return data_class.to_python(ctype_object)
+
+    @classmethod
+    def _init_python_map(cls):
+        """
+        Optimizes Python types→Ignite types map creation for speed.
+
+        Local imports seem inevitable here.
+        """
+        from pyignite.datatypes import (
+            LongObject, DoubleObject, String, BoolObject, Null, UUIDObject,
+            DateObject, TimeObject, DecimalObject,
+        )
+
+        cls._python_map = {
+            int: LongObject,
+            float: DoubleObject,
+            str: String,
+            bytes: String,
+            bool: BoolObject,
+            type(None): Null,
+            uuid.UUID: UUIDObject,
+            datetime: DateObject,
+            date: DateObject,
+            timedelta: TimeObject,
+            decimal.Decimal: DecimalObject,
+        }
+
+    @classmethod
+    def _init_python_array_map(cls):
+        """
+        Optimizes  Python types→Ignite array types map creation for speed.
+        """
+        from pyignite.datatypes import (
+            LongArrayObject, DoubleArrayObject, StringArrayObject,
+            BoolArrayObject, UUIDArrayObject, DateArrayObject, TimeArrayObject,
+            DecimalArrayObject,
+        )
+
+        cls._python_array_map = {
+            int: LongArrayObject,
+            float: DoubleArrayObject,
+            str: StringArrayObject,
+            bytes: StringArrayObject,
+            bool: BoolArrayObject,
+            uuid.UUID: UUIDArrayObject,
+            datetime: DateArrayObject,
+            date: DateArrayObject,
+            timedelta: TimeArrayObject,
+            decimal.Decimal: DecimalArrayObject,
+        }
+
+    @classmethod
+    def map_python_type(cls, value):
+        from pyignite.datatypes import (
+            MapObject, ObjectArrayObject, BinaryObject,
+        )
+
+        if cls._python_map is None:
+            cls._init_python_map()
+        if cls._python_array_map is None:
+            cls._init_python_array_map()
+
+        value_type = type(value)
+        if is_iterable(value) and value_type is not str:
+            value_subtype = cls.get_subtype(value)
+            if value_subtype in cls._python_array_map:
+                return cls._python_array_map[value_subtype]
+
+            # a little heuristics (order may be important)
+            if all([
+                value_subtype is None,
+                len(value) == 2,
+                isinstance(value[0], int),
+                isinstance(value[1], dict),
+            ]):
+                return MapObject
+
+            if all([
+                value_subtype is None,
+                len(value) == 2,
+                isinstance(value[0], int),
+                is_iterable(value[1]),
+            ]):
+                return ObjectArrayObject
+
+            raise TypeError(
+                'Type `array of {}` is invalid'.format(value_subtype)
+            )
+
+        if is_binary(value):
+            return BinaryObject
+
+        if value_type in cls._python_map:
+            return cls._python_map[value_type]
+        raise TypeError(
+            'Type `{}` is invalid.'.format(value_type)
+        )
+
+    @classmethod
+    def from_python(cls, value):
+        return cls.map_python_type(value).from_python(value)
+
+
+@attr.s
+class AnyDataArray(AnyDataObject):
+    """
+    Sequence of AnyDataObjects, payload-only.
+    """
+    counter_type = attr.ib(default=ctypes.c_int)
+
+    def build_header(self):
+        return type(
+            self.__class__.__name__+'Header',
+            (ctypes.LittleEndianStructure,),
+            {
+                '_pack_': 1,
+                '_fields_': [
+                    ('length', self.counter_type),
+                ],
+            }
+        )
+
+    def parse(self, client: 'Client'):
+        header_class = self.build_header()
+        buffer = client.recv(ctypes.sizeof(header_class))
+        header = header_class.from_buffer_copy(buffer)
+        fields = []
+
+        for i in range(header.length):
+            c_type, buffer_fragment = super().parse(client)
+            buffer += buffer_fragment
+            fields.append(('element_{}'.format(i), c_type))
+
+        final_class = type(
+            self.__class__.__name__,
+            (header_class,),
+            {
+                '_pack_': 1,
+                '_fields_': fields,
+            }
+        )
+        return final_class, buffer
+
+    @classmethod
+    def to_python(cls, ctype_object, *args, **kwargs):
+        result = []
+        for i in range(ctype_object.length):
+            result.append(
+                super().to_python(
+                    getattr(ctype_object, 'element_{}'.format(i)),
+                    *args, **kwargs
+                )
+            )
+        return result
+
+    def from_python(self, value):
+        header_class = self.build_header()
+        header = header_class()
+
+        try:
+            length = len(value)
+        except TypeError:
+            value = [value]
+            length = 1
+        header.length = length
+        buffer = bytes(header)
+
+        for x in value:
+            if is_hinted(x):
+                buffer += x[1].from_python(x[0])
+            else:
+                buffer += super().from_python(x)
+        return buffer

http://git-wip-us.apache.org/repos/asf/ignite/blob/7e547b13/modules/platforms/python/pyignite/datatypes/key_value.py
----------------------------------------------------------------------
diff --git a/modules/platforms/python/pyignite/datatypes/key_value.py b/modules/platforms/python/pyignite/datatypes/key_value.py
new file mode 100644
index 0000000..0f21ac6
--- /dev/null
+++ b/modules/platforms/python/pyignite/datatypes/key_value.py
@@ -0,0 +1,24 @@
+# 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.
+
+from .primitive_arrays import ByteArray
+
+
+class PeekModes(ByteArray):
+
+    ALL = 0
+    NEAR = 1
+    PRIMARY = 2
+    BACKUP = 3

http://git-wip-us.apache.org/repos/asf/ignite/blob/7e547b13/modules/platforms/python/pyignite/datatypes/null_object.py
----------------------------------------------------------------------
diff --git a/modules/platforms/python/pyignite/datatypes/null_object.py b/modules/platforms/python/pyignite/datatypes/null_object.py
new file mode 100644
index 0000000..9fa1e8f
--- /dev/null
+++ b/modules/platforms/python/pyignite/datatypes/null_object.py
@@ -0,0 +1,63 @@
+# 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.
+
+"""
+Null object.
+
+There can't be null type, because null payload takes exactly 0 bytes.
+"""
+
+import ctypes
+
+from .type_codes import TC_NULL
+
+
+__all__ = ['Null']
+
+
+class Null:
+    default = None
+    pythonic = type(None)
+    _object_c_type = None
+
+    @classmethod
+    def build_c_type(cls):
+        if cls._object_c_type is None:
+            cls._object_c_type = type(
+                cls.__name__,
+                (ctypes.LittleEndianStructure,),
+                {
+                    '_pack_': 1,
+                    '_fields_': [
+                        ('type_code', ctypes.c_byte),
+                    ],
+                },
+            )
+        return cls._object_c_type
+
+    @classmethod
+    def parse(cls, client: 'Client'):
+        buffer = client.recv(ctypes.sizeof(ctypes.c_byte))
+        data_type = cls.build_c_type()
+        return data_type, buffer
+
+    @staticmethod
+    def to_python(*args, **kwargs):
+        return None
+
+    @staticmethod
+    def from_python(*args):
+        return TC_NULL
+

http://git-wip-us.apache.org/repos/asf/ignite/blob/7e547b13/modules/platforms/python/pyignite/datatypes/primitive.py
----------------------------------------------------------------------
diff --git a/modules/platforms/python/pyignite/datatypes/primitive.py b/modules/platforms/python/pyignite/datatypes/primitive.py
new file mode 100644
index 0000000..73f096d
--- /dev/null
+++ b/modules/platforms/python/pyignite/datatypes/primitive.py
@@ -0,0 +1,105 @@
+# 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.
+
+import ctypes
+
+from pyignite.constants import *
+
+
+__all__ = [
+    'Primitive',
+    'Byte', 'Short', 'Int', 'Long', 'Float', 'Double', 'Char', 'Bool',
+]
+
+
+class Primitive:
+    """
+    Ignite primitive type. Base type for the following types:
+
+    - Byte,
+    - Short,
+    - Int,
+    - Long,
+    - Float,
+    - Double,
+    - Char,
+    - Bool.
+    """
+
+    c_type = None
+
+    @classmethod
+    def parse(cls, client: 'Client'):
+        return cls.c_type, client.recv(ctypes.sizeof(cls.c_type))
+
+    @staticmethod
+    def to_python(ctype_object, *args, **kwargs):
+        return ctype_object
+
+    @classmethod
+    def from_python(cls, value):
+        return bytes(cls.c_type(value))
+
+
+class Byte(Primitive):
+    c_type = ctypes.c_byte
+
+
+class Short(Primitive):
+    c_type = ctypes.c_short
+
+
+class Int(Primitive):
+    c_type = ctypes.c_int
+
+
+class Long(Primitive):
+    c_type = ctypes.c_long
+
+
+class Float(Primitive):
+    c_type = ctypes.c_float
+
+
+class Double(Primitive):
+    c_type = ctypes.c_double
+
+
+class Char(Primitive):
+    c_type = ctypes.c_short
+
+    @classmethod
+    def to_python(cls, ctype_object, *args, **kwargs):
+        return ctype_object.value.to_bytes(
+            ctypes.sizeof(cls.c_type),
+            byteorder=PROTOCOL_BYTE_ORDER
+        ).decode(PROTOCOL_CHAR_ENCODING)
+
+    @classmethod
+    def from_python(cls, value):
+        if type(value) is str:
+            value = value.encode(PROTOCOL_CHAR_ENCODING)
+        # assuming either a bytes or an integer
+        if type(value) is bytes:
+            value = int.from_bytes(value, byteorder=PROTOCOL_BYTE_ORDER)
+        # assuming a valid integer
+        return value.to_bytes(
+            ctypes.sizeof(cls.c_type),
+            byteorder=PROTOCOL_BYTE_ORDER
+        )
+
+
+class Bool(Primitive):
+    c_type = ctypes.c_bool

http://git-wip-us.apache.org/repos/asf/ignite/blob/7e547b13/modules/platforms/python/pyignite/datatypes/primitive_arrays.py
----------------------------------------------------------------------
diff --git a/modules/platforms/python/pyignite/datatypes/primitive_arrays.py b/modules/platforms/python/pyignite/datatypes/primitive_arrays.py
new file mode 100644
index 0000000..83a2b4c
--- /dev/null
+++ b/modules/platforms/python/pyignite/datatypes/primitive_arrays.py
@@ -0,0 +1,207 @@
+# 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.
+
+import ctypes
+
+from pyignite.constants import *
+from .primitive import *
+from .type_codes import *
+
+
+__all__ = [
+    'ByteArray', 'ByteArrayObject', 'ShortArray', 'ShortArrayObject',
+    'IntArray', 'IntArrayObject', 'LongArray', 'LongArrayObject',
+    'FloatArray', 'FloatArrayObject', 'DoubleArray', 'DoubleArrayObject',
+    'CharArray', 'CharArrayObject', 'BoolArray', 'BoolArrayObject',
+]
+
+
+class PrimitiveArray:
+    """
+    Base class for array of primitives. Payload-only.
+    """
+    primitive_type = None
+    type_code = None
+
+    @classmethod
+    def build_header_class(cls):
+        return type(
+            cls.__name__+'Header',
+            (ctypes.LittleEndianStructure,),
+            {
+                '_pack_': 1,
+                '_fields_': [
+                    ('length', ctypes.c_int),
+                ],
+            }
+        )
+
+    @classmethod
+    def parse(cls, client: 'Client'):
+        header_class = cls.build_header_class()
+        buffer = client.recv(ctypes.sizeof(header_class))
+        header = header_class.from_buffer_copy(buffer)
+        final_class = type(
+            cls.__name__,
+            (header_class,),
+            {
+                '_pack_': 1,
+                '_fields_': [
+                    ('data', cls.primitive_type.c_type * header.length),
+                ],
+            }
+        )
+        buffer += client.recv(
+            ctypes.sizeof(final_class) - ctypes.sizeof(header_class)
+        )
+        return final_class, buffer
+
+    @classmethod
+    def to_python(cls, ctype_object, *args, **kwargs):
+        result = []
+        for i in range(ctype_object.length):
+            result.append(ctype_object.data[i])
+        return result
+
+    @classmethod
+    def from_python(cls, value):
+        header_class = cls.build_header_class()
+        header = header_class()
+        if hasattr(header, 'type_code'):
+            header.type_code = int.from_bytes(
+                cls.type_code,
+                byteorder=PROTOCOL_BYTE_ORDER
+            )
+        length = len(value)
+        header.length = length
+        buffer = bytes(header)
+
+        for x in value:
+            buffer += cls.primitive_type.from_python(x)
+        return buffer
+
+
+class ByteArray(PrimitiveArray):
+    primitive_type = Byte
+    type_code = TC_BYTE_ARRAY
+
+
+class ShortArray(PrimitiveArray):
+    primitive_type = Short
+    type_code = TC_SHORT_ARRAY
+
+
+class IntArray(PrimitiveArray):
+    primitive_type = Int
+    type_code = TC_INT_ARRAY
+
+
+class LongArray(PrimitiveArray):
+    primitive_type = Long
+    type_code = TC_LONG_ARRAY
+
+
+class FloatArray(PrimitiveArray):
+    primitive_type = Float
+    type_code = TC_FLOAT_ARRAY
+
+
+class DoubleArray(PrimitiveArray):
+    primitive_type = Double
+    type_code = TC_DOUBLE_ARRAY
+
+
+class CharArray(PrimitiveArray):
+    primitive_type = Char
+    type_code = TC_CHAR_ARRAY
+
+
+class BoolArray(PrimitiveArray):
+    primitive_type = Bool
+    type_code = TC_BOOL_ARRAY
+
+
+class PrimitiveArrayObject(PrimitiveArray):
+    """
+    Base class for primitive array object. Type code plus payload.
+    """
+    pythonic = list
+    default = []
+
+    @classmethod
+    def build_header_class(cls):
+        return type(
+            cls.__name__+'Header',
+            (ctypes.LittleEndianStructure,),
+            {
+                '_pack_': 1,
+                '_fields_': [
+                    ('type_code', ctypes.c_byte),
+                    ('length', ctypes.c_int),
+                ],
+            }
+        )
+
+
+class ByteArrayObject(PrimitiveArrayObject):
+    primitive_type = Byte
+    type_code = TC_BYTE_ARRAY
+
+
+class ShortArrayObject(PrimitiveArrayObject):
+    primitive_type = Short
+    type_code = TC_SHORT_ARRAY
+
+
+class IntArrayObject(PrimitiveArrayObject):
+    primitive_type = Int
+    type_code = TC_INT_ARRAY
+
+
+class LongArrayObject(PrimitiveArrayObject):
+    primitive_type = Long
+    type_code = TC_LONG_ARRAY
+
+
+class FloatArrayObject(PrimitiveArrayObject):
+    primitive_type = Float
+    type_code = TC_FLOAT_ARRAY
+
+
+class DoubleArrayObject(PrimitiveArrayObject):
+    primitive_type = Double
+    type_code = TC_DOUBLE_ARRAY
+
+
+class CharArrayObject(PrimitiveArrayObject):
+    primitive_type = Char
+    type_code = TC_CHAR_ARRAY
+
+    @classmethod
+    def to_python(cls, ctype_object, *args, **kwargs):
+        values = super().to_python(ctype_object, *args, **kwargs)
+        return [
+            v.to_bytes(
+                ctypes.sizeof(cls.primitive_type.c_type),
+                byteorder=PROTOCOL_BYTE_ORDER
+            ).decode(
+                PROTOCOL_CHAR_ENCODING
+            ) for v in values
+        ]
+
+
+class BoolArrayObject(PrimitiveArrayObject):
+    primitive_type = Bool
+    type_code = TC_BOOL_ARRAY

http://git-wip-us.apache.org/repos/asf/ignite/blob/7e547b13/modules/platforms/python/pyignite/datatypes/primitive_objects.py
----------------------------------------------------------------------
diff --git a/modules/platforms/python/pyignite/datatypes/primitive_objects.py b/modules/platforms/python/pyignite/datatypes/primitive_objects.py
new file mode 100644
index 0000000..53b54b3
--- /dev/null
+++ b/modules/platforms/python/pyignite/datatypes/primitive_objects.py
@@ -0,0 +1,157 @@
+# 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.
+
+import ctypes
+
+from pyignite.constants import *
+from .type_codes import *
+
+
+__all__ = [
+    'DataObject', 'ByteObject', 'ShortObject', 'IntObject', 'LongObject',
+    'FloatObject', 'DoubleObject', 'CharObject', 'BoolObject',
+]
+
+
+class DataObject:
+    """
+    Base class for primitive data objects.
+
+    Primitive data objects are built of primitive data prepended by
+    the corresponding type code.
+    """
+
+    c_type = None
+    type_code = None
+    _object_c_type = None
+
+    @classmethod
+    def build_c_type(cls):
+        if cls._object_c_type is None:
+            cls._object_c_type = type(
+                cls.__name__,
+                (ctypes.LittleEndianStructure,),
+                {
+                    '_pack_': 1,
+                    '_fields_': [
+                        ('type_code', ctypes.c_byte),
+                        ('value', cls.c_type),
+                    ],
+                },
+            )
+        return cls._object_c_type
+
+    @classmethod
+    def parse(cls, client: 'Client'):
+        data_type = cls.build_c_type()
+        buffer = client.recv(ctypes.sizeof(data_type))
+        return data_type, buffer
+
+    @staticmethod
+    def to_python(ctype_object, *args, **kwargs):
+        return ctype_object.value
+
+    @classmethod
+    def from_python(cls, value):
+        data_type = cls.build_c_type()
+        data_object = data_type()
+        data_object.type_code = int.from_bytes(
+            cls.type_code,
+            byteorder=PROTOCOL_BYTE_ORDER
+        )
+        data_object.value = value
+        return bytes(data_object)
+
+
+class ByteObject(DataObject):
+    c_type = ctypes.c_byte
+    type_code = TC_BYTE
+    pythonic = int
+    default = 0
+
+
+class ShortObject(DataObject):
+    c_type = ctypes.c_short
+    type_code = TC_SHORT
+    pythonic = int
+    default = 0
+
+
+class IntObject(DataObject):
+    c_type = ctypes.c_int
+    type_code = TC_INT
+    pythonic = int
+    default = 0
+
+
+class LongObject(DataObject):
+    c_type = ctypes.c_long
+    type_code = TC_LONG
+    pythonic = int
+    default = 0
+
+
+class FloatObject(DataObject):
+    c_type = ctypes.c_float
+    type_code = TC_FLOAT
+    pythonic = float
+    default = 0.0
+
+
+class DoubleObject(DataObject):
+    c_type = ctypes.c_double
+    type_code = TC_DOUBLE
+    pythonic = float
+    default = 0.0
+
+
+class CharObject(DataObject):
+    """
+    This type is a little tricky. It stores character values in
+    UTF-16 Little-endian encoding. We have to encode/decode it
+    to/from UTF-8 to keep the coding hassle to minimum. Bear in mind
+    though: decoded character may take 1..4 bytes in UTF-8.
+    """
+    c_type = ctypes.c_short
+    type_code = TC_CHAR
+    pythonic = str
+    default = ' '
+
+    @classmethod
+    def to_python(cls, ctype_object, *args, **kwargs):
+        return ctype_object.value.to_bytes(
+            ctypes.sizeof(cls.c_type),
+            byteorder=PROTOCOL_BYTE_ORDER
+        ).decode(PROTOCOL_CHAR_ENCODING)
+
+    @classmethod
+    def from_python(cls, value):
+        if type(value) is str:
+            value = value.encode(PROTOCOL_CHAR_ENCODING)
+        # assuming either a bytes or an integer
+        if type(value) is bytes:
+            value = int.from_bytes(value, byteorder=PROTOCOL_BYTE_ORDER)
+        # assuming a valid integer
+        return cls.type_code + value.to_bytes(
+            ctypes.sizeof(cls.c_type),
+            byteorder=PROTOCOL_BYTE_ORDER
+        )
+
+
+class BoolObject(DataObject):
+    c_type = ctypes.c_bool
+    type_code = TC_BOOL
+    pythonic = bool
+    default = False

http://git-wip-us.apache.org/repos/asf/ignite/blob/7e547b13/modules/platforms/python/pyignite/datatypes/prop_codes.py
----------------------------------------------------------------------
diff --git a/modules/platforms/python/pyignite/datatypes/prop_codes.py b/modules/platforms/python/pyignite/datatypes/prop_codes.py
new file mode 100644
index 0000000..adea281
--- /dev/null
+++ b/modules/platforms/python/pyignite/datatypes/prop_codes.py
@@ -0,0 +1,51 @@
+# 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.
+
+PROP_NAME = 0
+PROP_CACHE_MODE = 1
+PROP_CACHE_ATOMICITY_MODE = 2
+PROP_BACKUPS_NUMBER = 3
+PROP_WRITE_SYNCHRONIZATION_MODE = 4
+PROP_COPY_ON_READ = 5
+PROP_READ_FROM_BACKUP = 6
+
+PROP_DATA_REGION_NAME = 100
+PROP_IS_ONHEAP_CACHE_ENABLED = 101
+
+PROP_QUERY_ENTITIES = 200
+PROP_QUERY_PARALLELISM = 201
+PROP_QUERY_DETAIL_METRIC_SIZE = 202
+PROP_SQL_SCHEMA = 203
+PROP_SQL_INDEX_INLINE_MAX_SIZE = 204
+PROP_SQL_ESCAPE_ALL = 205
+PROP_MAX_QUERY_ITERATORS = 206
+
+PROP_REBALANCE_MODE = 300
+PROP_REBALANCE_DELAY = 301
+PROP_REBALANCE_TIMEOUT = 302
+PROP_REBALANCE_BATCH_SIZE = 303
+PROP_REBALANCE_BATCHES_PREFETCH_COUNT = 304
+PROP_REBALANCE_ORDER = 305
+PROP_REBALANCE_THROTTLE = 306
+
+PROP_GROUP_NAME = 400
+PROP_CACHE_KEY_CONFIGURATION = 401
+PROP_DEFAULT_LOCK_TIMEOUT = 402
+PROP_MAX_CONCURRENT_ASYNC_OPERATIONS = 403
+PROP_PARTITION_LOSS_POLICY = 404
+PROP_EAGER_TTL = 405
+PROP_STATISTICS_ENABLED = 406
+
+PROP_INVALIDATE = -1

http://git-wip-us.apache.org/repos/asf/ignite/blob/7e547b13/modules/platforms/python/pyignite/datatypes/sql.py
----------------------------------------------------------------------
diff --git a/modules/platforms/python/pyignite/datatypes/sql.py b/modules/platforms/python/pyignite/datatypes/sql.py
new file mode 100644
index 0000000..533349e
--- /dev/null
+++ b/modules/platforms/python/pyignite/datatypes/sql.py
@@ -0,0 +1,23 @@
+# 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.
+
+from .primitive import Byte
+
+
+class StatementType(Byte):
+
+    ANY = 0
+    SELECT = 1
+    UPDATE = 2

http://git-wip-us.apache.org/repos/asf/ignite/blob/7e547b13/modules/platforms/python/pyignite/datatypes/standard.py
----------------------------------------------------------------------
diff --git a/modules/platforms/python/pyignite/datatypes/standard.py b/modules/platforms/python/pyignite/datatypes/standard.py
new file mode 100644
index 0000000..160e06e
--- /dev/null
+++ b/modules/platforms/python/pyignite/datatypes/standard.py
@@ -0,0 +1,713 @@
+# 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.
+
+import ctypes
+from datetime import date, datetime, time, timedelta
+import decimal
+import uuid
+
+from pyignite.constants import *
+from .type_codes import *
+from .null_object import Null
+
+
+__all__ = [
+    'String', 'DecimalObject', 'UUIDObject', 'TimestampObject', 'DateObject',
+    'TimeObject',
+
+    'StringArray', 'DecimalArray', 'UUIDArray', 'TimestampArray', 'DateArray',
+    'TimeArray',
+
+    'StringArrayObject', 'DecimalArrayObject', 'UUIDArrayObject',
+    'TimestampArrayObject', 'TimeArrayObject', 'DateArrayObject',
+
+    'EnumObject', 'EnumArray', 'EnumArrayObject', 'BinaryEnumObject',
+    'BinaryEnumArrayObject', 'ObjectArray',
+]
+
+
+class StandardObject:
+    type_code = None
+
+    @classmethod
+    def build_c_type(cls):
+        raise NotImplementedError('This object is generic')
+
+    @classmethod
+    def parse(cls, client: 'Client'):
+        tc_type = client.recv(ctypes.sizeof(ctypes.c_byte))
+
+        if tc_type == TC_NULL:
+            return Null.build_c_type(), tc_type
+
+        c_type = cls.build_c_type()
+        buffer = tc_type + client.recv(ctypes.sizeof(c_type) - len(tc_type))
+        return c_type, buffer
+
+
+class String:
+    """
+    Pascal-style string: `c_int` counter, followed by count*bytes.
+    UTF-8-encoded, so that one character may take 1 to 4 bytes.
+    """
+    type_code = TC_STRING
+    pythonic = str
+
+    @classmethod
+    def build_c_type(cls, length: int):
+        return type(
+            cls.__name__,
+            (ctypes.LittleEndianStructure,),
+            {
+                '_pack_': 1,
+                '_fields_': [
+                    ('type_code', ctypes.c_byte),
+                    ('length', ctypes.c_int),
+                    ('data', ctypes.c_char * length),
+                ],
+            },
+        )
+
+    @classmethod
+    def parse(cls, client: 'Client'):
+        tc_type = client.recv(ctypes.sizeof(ctypes.c_byte))
+        # String or Null
+        if tc_type == TC_NULL:
+            return Null.build_c_type(), tc_type
+
+        buffer = tc_type + client.recv(ctypes.sizeof(ctypes.c_int))
+        length = int.from_bytes(buffer[1:], byteorder=PROTOCOL_BYTE_ORDER)
+
+        data_type = cls.build_c_type(length)
+        buffer += client.recv(ctypes.sizeof(data_type) - len(buffer))
+
+        return data_type, buffer
+
+    @staticmethod
+    def to_python(ctype_object, *args, **kwargs):
+        length = getattr(ctype_object, 'length', None)
+        if length is None:
+            return None
+        elif length > 0:
+            return ctype_object.data.decode(PROTOCOL_STRING_ENCODING)
+        else:
+            return ''
+
+    @classmethod
+    def from_python(cls, value):
+        if value is None:
+            return Null.from_python()
+
+        if isinstance(value, str):
+            value = value.encode(PROTOCOL_STRING_ENCODING)
+        length = len(value)
+        data_type = cls.build_c_type(length)
+        data_object = data_type()
+        data_object.type_code = int.from_bytes(
+            cls.type_code,
+            byteorder=PROTOCOL_BYTE_ORDER
+        )
+        data_object.length = length
+        data_object.data = value
+        return bytes(data_object)
+
+
+class DecimalObject:
+    type_code = TC_DECIMAL
+    pythonic = decimal.Decimal
+    default = decimal.Decimal('0.00')
+
+    @classmethod
+    def build_c_header(cls):
+        return type(
+            cls.__name__,
+            (ctypes.LittleEndianStructure,),
+            {
+                '_pack_': 1,
+                '_fields_': [
+                    ('type_code', ctypes.c_byte),
+                    ('scale', ctypes.c_int),
+                    ('length', ctypes.c_int),
+                ],
+            }
+        )
+
+    @classmethod
+    def parse(cls, client: 'Client'):
+        tc_type = client.recv(ctypes.sizeof(ctypes.c_byte))
+        # Decimal or Null
+        if tc_type == TC_NULL:
+            return Null.build_c_type(), tc_type
+
+        header_class = cls.build_c_header()
+        buffer = tc_type + client.recv(
+            ctypes.sizeof(header_class)
+            - len(tc_type)
+        )
+        header = header_class.from_buffer_copy(buffer)
+        data_type = type(
+            cls.__name__,
+            (header_class,),
+            {
+                '_pack_': 1,
+                '_fields_': [
+                    ('data', ctypes.c_char * header.length),
+                ],
+            }
+        )
+        buffer += client.recv(
+            ctypes.sizeof(data_type)
+            - ctypes.sizeof(header_class)
+        )
+        return data_type, buffer
+
+    @classmethod
+    def to_python(cls, ctype_object, *args, **kwargs):
+        if getattr(ctype_object, 'length', None) is None:
+            return None
+
+        sign = 1 if ctype_object.data[0] & 0x80 else 0
+        data = bytes([ctype_object.data[0] & 0x7f]) + ctype_object.data[1:]
+        result = decimal.Decimal(data.decode(PROTOCOL_STRING_ENCODING))
+        # apply scale
+        result = (
+            result
+            * decimal.Decimal('10') ** decimal.Decimal(ctype_object.scale)
+        )
+        if sign:
+            # apply sign
+            result = -result
+        return result
+
+    @classmethod
+    def from_python(cls, value: decimal.Decimal):
+        if value is None:
+            return Null.from_python()
+
+        sign, digits, scale = value.normalize().as_tuple()
+        data = bytearray([ord('0') + digit for digit in digits])
+        if sign:
+            data[0] |= 0x80
+        else:
+            data[0] &= 0x7f
+        length = len(digits)
+        header_class = cls.build_c_header()
+        data_class = type(
+            cls.__name__,
+            (header_class,),
+            {
+                '_pack_': 1,
+                '_fields_': [
+                    ('data', ctypes.c_char * length),
+                ],
+            }
+        )
+        data_object = data_class()
+        data_object.type_code = int.from_bytes(
+            cls.type_code,
+            byteorder=PROTOCOL_BYTE_ORDER
+        )
+        data_object.length = length
+        data_object.scale = scale
+        data_object.data = bytes(data)
+        return bytes(data_object)
+
+
+class UUIDObject(StandardObject):
+    """
+    Universally unique identifier (UUID), aka Globally unique identifier
+    (GUID). Payload takes up 16 bytes.
+    """
+    type_code = TC_UUID
+    _object_c_type = None
+
+    @classmethod
+    def build_c_type(cls):
+        if cls._object_c_type is None:
+            cls._object_c_type = type(
+                cls.__name__,
+                (ctypes.LittleEndianStructure,),
+                {
+                    '_pack_': 1,
+                    '_fields_': [
+                        ('type_code', ctypes.c_byte),
+                        ('value', ctypes.c_byte * 16),
+                    ],
+                }
+            )
+        return cls._object_c_type
+
+    @classmethod
+    def from_python(cls, value: uuid.UUID):
+        data_type = cls.build_c_type()
+        data_object = data_type()
+        data_object.type_code = int.from_bytes(
+            cls.type_code,
+            byteorder=PROTOCOL_BYTE_ORDER
+        )
+        for i, byte in enumerate(bytearray(value.bytes)):
+            data_object.value[i] = byte
+        return bytes(data_object)
+
+    @classmethod
+    def to_python(cls, ctypes_object, *args, **kwargs):
+        if ctypes_object.type_code == int.from_bytes(
+            TC_NULL,
+            byteorder=PROTOCOL_BYTE_ORDER
+        ):
+            return None
+        return uuid.UUID(bytes=bytes(ctypes_object.value))
+
+
+class TimestampObject(StandardObject):
+    """
+    A signed integer number of milliseconds past 1 Jan 1970, aka Epoch
+    (8 bytes long integer), plus the delta in nanoseconds (4 byte integer,
+    only 0..999 range used).
+
+    The accuracy is ridiculous. For instance, common HPETs have
+    less than 10ms accuracy. Therefore no ns range calculations is made;
+    `epoch` and `fraction` stored separately and represented as
+    tuple(datetime.datetime, integer).
+    """
+    type_code = TC_TIMESTAMP
+    pythonic = tuple
+    default = (datetime(1970, 1, 1), 0)
+    _object_c_type = None
+
+    @classmethod
+    def build_c_type(cls):
+        if cls._object_c_type is None:
+            cls._object_c_type = type(
+                cls.__name__,
+                (ctypes.LittleEndianStructure,),
+                {
+                    '_pack_': 1,
+                    '_fields_': [
+                        ('type_code', ctypes.c_byte),
+                        ('epoch', ctypes.c_long),
+                        ('fraction', ctypes.c_int),
+                    ],
+                }
+            )
+        return cls._object_c_type
+
+    @classmethod
+    def from_python(cls, value: tuple):
+        if value is None:
+            return Null.from_python()
+        data_type = cls.build_c_type()
+        data_object = data_type()
+        data_object.type_code = int.from_bytes(
+            cls.type_code,
+            byteorder=PROTOCOL_BYTE_ORDER
+        )
+        data_object.epoch = int(value[0].timestamp() * 1000)
+        data_object.fraction = value[1]
+        return bytes(data_object)
+
+    @classmethod
+    def to_python(cls, ctypes_object, *args, **kwargs):
+        if ctypes_object.type_code == int.from_bytes(
+            TC_NULL,
+            byteorder=PROTOCOL_BYTE_ORDER
+        ):
+            return None
+        return (
+            datetime.fromtimestamp(ctypes_object.epoch/1000),
+            ctypes_object.fraction
+        )
+
+
+class DateObject(StandardObject):
+    """
+    A signed integer number of milliseconds past 1 Jan 1970, aka Epoch
+    (8 bytes long integer).
+
+    Represented as a naive datetime.datetime in Python.
+    """
+    type_code = TC_DATE
+    pythonic = datetime
+    default = datetime(1970, 1, 1)
+    _object_c_type = None
+
+    @classmethod
+    def build_c_type(cls):
+        if cls._object_c_type is None:
+            cls._object_c_type = type(
+                cls.__name__,
+                (ctypes.LittleEndianStructure,),
+                {
+                    '_pack_': 1,
+                    '_fields_': [
+                        ('type_code', ctypes.c_byte),
+                        ('epoch', ctypes.c_long),
+                    ],
+                }
+            )
+        return cls._object_c_type
+
+    @classmethod
+    def from_python(cls, value: [date, datetime]):
+        if value is None:
+            return Null.from_python()
+        if type(value) is date:
+            value = datetime.combine(value, time())
+        data_type = cls.build_c_type()
+        data_object = data_type()
+        data_object.type_code = int.from_bytes(
+            cls.type_code,
+            byteorder=PROTOCOL_BYTE_ORDER
+        )
+        data_object.epoch = int(value.timestamp() * 1000)
+        return bytes(data_object)
+
+    @classmethod
+    def to_python(cls, ctypes_object, *args, **kwargs):
+        if ctypes_object.type_code == int.from_bytes(
+            TC_NULL,
+            byteorder=PROTOCOL_BYTE_ORDER
+        ):
+            return None
+        return datetime.fromtimestamp(ctypes_object.epoch/1000)
+
+
+class TimeObject(StandardObject):
+    """
+    Time of the day as a number of milliseconds since midnight.
+
+    Represented as a datetime.timedelta in Python.
+    """
+    type_code = TC_TIME
+    pythonic = timedelta
+    default = timedelta()
+    _object_c_type = None
+
+    @classmethod
+    def build_c_type(cls):
+        if cls._object_c_type is None:
+            cls._object_c_type = type(
+                cls.__name__,
+                (ctypes.LittleEndianStructure,),
+                {
+                    '_pack_': 1,
+                    '_fields_': [
+                        ('type_code', ctypes.c_byte),
+                        ('value', ctypes.c_long),
+                    ],
+                }
+            )
+        return cls._object_c_type
+
+    @classmethod
+    def from_python(cls, value: timedelta):
+        if value is None:
+            return Null.from_python()
+        data_type = cls.build_c_type()
+        data_object = data_type()
+        data_object.type_code = int.from_bytes(
+            cls.type_code,
+            byteorder=PROTOCOL_BYTE_ORDER
+        )
+        data_object.value = int(value.total_seconds() * 1000)
+        return bytes(data_object)
+
+    @classmethod
+    def to_python(cls, ctypes_object, *args, **kwargs):
+        if ctypes_object.type_code == int.from_bytes(
+            TC_NULL,
+            byteorder=PROTOCOL_BYTE_ORDER
+        ):
+            return None
+        return timedelta(milliseconds=ctypes_object.value)
+
+
+class EnumObject(StandardObject):
+    """
+    Two integers used as the ID of the enumeration type, and its value.
+
+    This type itself is useless in Python, but can be used for interoperability
+    (using language-specific type serialization is a good way to kill the
+    interoperability though), so it represented by tuple(int, int) in Python.
+    """
+    type_code = TC_ENUM
+    _object_c_type = None
+
+    @classmethod
+    def build_c_type(cls):
+        if cls._object_c_type is None:
+            cls._object_c_type = type(
+                cls.__name__,
+                (ctypes.LittleEndianStructure,),
+                {
+                    '_pack_': 1,
+                    '_fields_': [
+                        ('type_code', ctypes.c_byte),
+                        ('type_id', ctypes.c_int),
+                        ('ordinal', ctypes.c_int),
+                    ],
+                }
+            )
+        return cls._object_c_type
+
+    @classmethod
+    def from_python(cls, value: tuple):
+        if value is None:
+            return Null.from_python()
+
+        data_type = cls.build_c_type()
+        data_object = data_type()
+        data_object.type_code = int.from_bytes(
+            cls.type_code,
+            byteorder=PROTOCOL_BYTE_ORDER
+        )
+        if value is None:
+            return Null.from_python(value)
+        data_object.type_id, data_object.ordinal = value
+        return bytes(data_object)
+
+    @classmethod
+    def to_python(cls, ctypes_object, *args, **kwargs):
+        if ctypes_object.type_code == int.from_bytes(
+            TC_NULL,
+            byteorder=PROTOCOL_BYTE_ORDER
+        ):
+            return None
+        return ctypes_object.type_id, ctypes_object.ordinal
+
+
+class BinaryEnumObject(EnumObject):
+    """
+    Another way of representing the enum type. Same, but different.
+    """
+    type_code = TC_BINARY_ENUM
+
+
+class StandardArray:
+    """
+    Base class for array of primitives. Payload-only.
+    """
+    standard_type = None
+    type_code = None
+
+    @classmethod
+    def build_header_class(cls):
+        return type(
+            cls.__name__+'Header',
+            (ctypes.LittleEndianStructure,),
+            {
+                '_pack_': 1,
+                '_fields_': [
+                    ('length', ctypes.c_int),
+                ],
+            }
+        )
+
+    @classmethod
+    def parse(cls, client: 'Client'):
+        header_class = cls.build_header_class()
+        buffer = client.recv(ctypes.sizeof(header_class))
+        header = header_class.from_buffer_copy(buffer)
+        fields = []
+        for i in range(header.length):
+            c_type, buffer_fragment = cls.standard_type.parse(client)
+            buffer += buffer_fragment
+            fields.append(('element_{}'.format(i), c_type))
+
+        final_class = type(
+            cls.__name__,
+            (header_class,),
+            {
+                '_pack_': 1,
+                '_fields_': fields,
+            }
+        )
+        return final_class, buffer
+
+    @classmethod
+    def to_python(cls, ctype_object, *args, **kwargs):
+        result = []
+        for i in range(ctype_object.length):
+            result.append(
+                cls.standard_type.to_python(
+                    getattr(ctype_object, 'element_{}'.format(i)),
+                    *args, **kwargs
+                )
+            )
+        return result
+
+    @classmethod
+    def from_python(cls, value):
+        header_class = cls.build_header_class()
+        header = header_class()
+        if hasattr(header, 'type_code'):
+            header.type_code = int.from_bytes(
+                cls.type_code,
+                byteorder=PROTOCOL_BYTE_ORDER
+            )
+        length = len(value)
+        header.length = length
+        buffer = bytes(header)
+
+        for x in value:
+            buffer += cls.standard_type.from_python(x)
+        return buffer
+
+
+class StringArray(StandardArray):
+    """
+    Array of Pascal-like strings. Payload-only, i.e. no `type_code` field
+    in binary representation.
+
+    List(str) in Python.
+    """
+    standard_type = String
+
+
+class DecimalArray(StandardArray):
+    standard_type = DecimalObject
+
+
+class UUIDArray(StandardArray):
+    standard_type = UUIDObject
+
+
+class TimestampArray(StandardArray):
+    standard_type = TimestampObject
+
+
+class DateArray(StandardArray):
+    standard_type = DateObject
+
+
+class TimeArray(StandardArray):
+    standard_type = TimeObject
+
+
+class EnumArray(StandardArray):
+    standard_type = EnumObject
+
+
+class StandardArrayObject(StandardArray):
+    pythonic = list
+    default = []
+
+    @classmethod
+    def build_header_class(cls):
+        return type(
+            cls.__name__+'Header',
+            (ctypes.LittleEndianStructure,),
+            {
+                '_pack_': 1,
+                '_fields_': [
+                    ('type_code', ctypes.c_byte),
+                    ('length', ctypes.c_int),
+                ],
+            }
+        )
+
+
+class StringArrayObject(StandardArrayObject):
+    """ List of strings. """
+    standard_type = String
+    type_code = TC_STRING_ARRAY
+
+
+class DecimalArrayObject(StandardArrayObject):
+    """ List of decimal.Decimal objects. """
+    standard_type = DecimalObject
+    type_code = TC_DECIMAL_ARRAY
+
+
+class UUIDArrayObject(StandardArrayObject):
+    """ Translated into Python as a list(uuid.UUID)"""
+    standard_type = UUIDObject
+    type_code = TC_UUID_ARRAY
+
+
+class TimestampArrayObject(StandardArrayObject):
+    """
+    Translated into Python as a list of (datetime.datetime, integer) tuples.
+    """
+    standard_type = TimestampObject
+    type_code = TC_TIMESTAMP_ARRAY
+
+
+class DateArrayObject(StandardArrayObject):
+    """ List of datetime.datetime type values. """
+    standard_type = DateObject
+    type_code = TC_DATE_ARRAY
+
+
+class TimeArrayObject(StandardArrayObject):
+    """ List of datetime.timedelta type values. """
+    standard_type = TimeObject
+    type_code = TC_TIME_ARRAY
+
+
+class EnumArrayObject(StandardArrayObject):
+    """
+    Array of (int, int) tuples, plus it holds a `type_id` in its header.
+    The only `type_id` value of -1 (user type) works from Python perspective.
+    """
+    standard_type = EnumObject
+    type_code = TC_ENUM_ARRAY
+
+    @classmethod
+    def build_header_class(cls):
+        return type(
+            cls.__name__+'Header',
+            (ctypes.LittleEndianStructure,),
+            {
+                '_pack_': 1,
+                '_fields_': [
+                    ('type_code', ctypes.c_byte),
+                    ('type_id', ctypes.c_int),
+                    ('length', ctypes.c_int),
+                ],
+            }
+        )
+
+    @classmethod
+    def from_python(cls, value):
+        type_id, value = value
+        header_class = cls.build_header_class()
+        header = header_class()
+        if hasattr(header, 'type_code'):
+            header.type_code = int.from_bytes(
+                cls.type_code,
+                byteorder=PROTOCOL_BYTE_ORDER
+            )
+        length = len(value)
+        header.length = length
+        header.type_id = type_id
+        buffer = bytes(header)
+
+        for x in value:
+            buffer += cls.standard_type.from_python(x)
+        return buffer
+
+    @classmethod
+    def to_python(cls, ctype_object, *args, **kwargs):
+        type_id = ctype_object.type_id
+        return type_id, super().to_python(ctype_object, *args, **kwargs)
+
+
+class BinaryEnumArrayObject(EnumArrayObject):
+    standard_type = BinaryEnumObject
+
+
+class ObjectArray(EnumArrayObject):
+    standard_type = BinaryEnumObject

http://git-wip-us.apache.org/repos/asf/ignite/blob/7e547b13/modules/platforms/python/pyignite/datatypes/type_codes.py
----------------------------------------------------------------------
diff --git a/modules/platforms/python/pyignite/datatypes/type_codes.py b/modules/platforms/python/pyignite/datatypes/type_codes.py
new file mode 100644
index 0000000..d5e8dd4
--- /dev/null
+++ b/modules/platforms/python/pyignite/datatypes/type_codes.py
@@ -0,0 +1,57 @@
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+TC_BYTE = b'\x01'
+TC_SHORT = b'\x02'
+TC_INT = b'\x03'
+TC_LONG = b'\x04'
+TC_FLOAT = b'\x05'
+TC_DOUBLE = b'\x06'
+TC_CHAR = b'\x07'
+TC_BOOL = b'\x08'
+TC_STRING = b'\x09'
+TC_UUID = b'\x0a'
+TC_DATE = b'\x0b'
+TC_BYTE_ARRAY = b'\x0c'
+TC_SHORT_ARRAY = b'\x0d'
+TC_INT_ARRAY = b'\x0e'
+TC_LONG_ARRAY = b'\x0f'
+TC_FLOAT_ARRAY = b'\x10'
+TC_DOUBLE_ARRAY = b'\x11'
+TC_CHAR_ARRAY = b'\x12'
+TC_BOOL_ARRAY = b'\x13'
+TC_STRING_ARRAY = b'\x14'
+TC_UUID_ARRAY = b'\x15'
+TC_DATE_ARRAY = b'\x16'
+TC_OBJECT_ARRAY = b'\x17'
+TC_COLLECTION = b'\x18'
+TC_MAP = b'\x19'
+
+TC_ARRAY_WRAPPED_OBJECTS = b'\x1b'
+
+TC_ENUM = b'\x1c'
+TC_ENUM_ARRAY = b'\x1d'
+
+TC_DECIMAL = b'\x1e'
+TC_DECIMAL_ARRAY = b'\x1f'
+TC_TIMESTAMP = b'\x21'
+TC_TIMESTAMP_ARRAY = b'\x22'
+
+TC_TIME = b'\x24'
+TC_TIME_ARRAY = b'\x25'
+TC_BINARY_ENUM = b'\x26'
+
+TC_NULL = b'\x65'
+TC_COMPLEX_OBJECT = b'\x67'

http://git-wip-us.apache.org/repos/asf/ignite/blob/7e547b13/modules/platforms/python/pyignite/exceptions.py
----------------------------------------------------------------------
diff --git a/modules/platforms/python/pyignite/exceptions.py b/modules/platforms/python/pyignite/exceptions.py
new file mode 100644
index 0000000..2bc5996
--- /dev/null
+++ b/modules/platforms/python/pyignite/exceptions.py
@@ -0,0 +1,80 @@
+# 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.
+
+from socket import error as SocketError
+
+
+class ParseError(Exception):
+    """
+    This exception is raised, when `pyignite` is unable to build a query to,
+    or parse a response from, Ignite node.
+    """
+    pass
+
+
+class HandshakeError(SocketError):
+    """
+    This exception is raised on Ignite binary protocol handshake failure,
+    as defined in
+    https://apacheignite.readme.io/docs/binary-client-protocol#section-handshake
+    """
+    pass
+
+
+class ReconnectError(Exception):
+    """
+    This exception is raised by `Client.reconnect` method, when no more
+    nodes are left to connect to. It is not meant to be an error, but rather
+    a flow control tool, similar to `StopIteration`.
+    """
+    pass
+
+
+class ParameterError(Exception):
+    """
+    This exception represents the parameter validation error in any `pyignite`
+    method.
+    """
+    pass
+
+
+class CacheError(Exception):
+    """
+    This exception is raised, whenever any remote Thin client operation
+    returns an error.
+    """
+    pass
+
+
+class BinaryTypeError(CacheError):
+    """
+    A remote error in operation with Complex Object registry.
+    """
+    pass
+
+
+class CacheCreationError(CacheError):
+    """
+    This exception is raised, when any complex operation failed
+    on cache creation phase.
+    """
+    pass
+
+
+class SQLError(CacheError):
+    """
+    An error in SQL query.
+    """
+    pass