You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2015/11/11 09:25:20 UTC

[24/67] [abbrv] ignite git commit: Merged IGNITE-950-new into IGNITE-1282

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/IgnitePortables.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgnitePortables.java b/modules/core/src/main/java/org/apache/ignite/IgnitePortables.java
deleted file mode 100644
index 1c63df7..0000000
--- a/modules/core/src/main/java/org/apache/ignite/IgnitePortables.java
+++ /dev/null
@@ -1,370 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite;
-
-import java.sql.Timestamp;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.TreeMap;
-import java.util.UUID;
-import org.apache.ignite.marshaller.portable.PortableMarshaller;
-import org.apache.ignite.portable.PortableBuilder;
-import org.apache.ignite.portable.PortableException;
-import org.apache.ignite.portable.PortableIdMapper;
-import org.apache.ignite.portable.PortableMarshalAware;
-import org.apache.ignite.portable.PortableMetadata;
-import org.apache.ignite.portable.PortableObject;
-import org.apache.ignite.portable.PortableSerializer;
-import org.apache.ignite.portable.PortableTypeConfiguration;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Defines portable objects functionality. With portable objects you are able to:
- * <ul>
- * <li>Seamlessly interoperate between Java, .NET, and C++.</li>
- * <li>Make any object portable with zero code change to your existing code.</li>
- * <li>Nest portable objects within each other.</li>
- * <li>Automatically handle {@code circular} or {@code null} references.</li>
- * <li>Automatically convert collections and maps between Java, .NET, and C++.</li>
- * <li>
- *      Optionally avoid deserialization of objects on the server side
- *      (objects are stored in {@link PortableObject} format).
- * </li>
- * <li>Avoid need to have concrete class definitions on the server side.</li>
- * <li>Dynamically change structure of the classes without having to restart the cluster.</li>
- * <li>Index into portable objects for querying purposes.</li>
- * </ul>
- * <h1 class="header">Working With Portables Directly</h1>
- * Once an object is defined as portable,
- * Ignite will always store it in memory in the portable (i.e. binary) format.
- * User can choose to work either with the portable format or with the deserialized form
- * (assuming that class definitions are present in the classpath).
- * <p>
- * To work with the portable format directly, user should create a special cache projection
- * using IgniteCache.withKeepPortable() method and then retrieve individual fields as needed:
- * <pre name=code class=java>
- * IgniteCache&lt;PortableObject, PortableObject&gt; prj = cache.withKeepPortable();
- *
- * // Convert instance of MyKey to portable format.
- * // We could also use PortableBuilder to create the key in portable format directly.
- * PortableObject key = grid.portables().toPortable(new MyKey());
- *
- * PortableObject val = prj.get(key);
- *
- * String field = val.field("myFieldName");
- * </pre>
- * Alternatively, if we have class definitions in the classpath, we may choose to work with deserialized
- * typed objects at all times. In this case we do incur the deserialization cost. However, if
- * {@link PortableMarshaller#isKeepDeserialized()} is {@code true} then Ignite will only deserialize on the first access
- * and will cache the deserialized object, so it does not have to be deserialized again:
- * <pre name=code class=java>
- * IgniteCache&lt;MyKey.class, MyValue.class&gt; cache = grid.cache(null);
- *
- * MyValue val = cache.get(new MyKey());
- *
- * // Normal java getter.
- * String fieldVal = val.getMyFieldName();
- * </pre>
- * If we used, for example, one of the automatically handled portable types for a key, like integer,
- * and still wanted to work with binary portable format for values, then we would declare cache projection
- * as follows:
- * <pre name=code class=java>
- * IgniteCache&lt;Integer.class, PortableObject&gt; prj = cache.withKeepPortable();
- * </pre>
- * <h1 class="header">Automatic Portable Types</h1>
- * Note that only portable classes are converted to {@link PortableObject} format. Following
- * classes are never converted (e.g., {@link #toPortable(Object)} method will return original
- * object, and instances of these classes will be stored in cache without changes):
- * <ul>
- *     <li>All primitives (byte, int, ...) and there boxed versions (Byte, Integer, ...)</li>
- *     <li>Arrays of primitives (byte[], int[], ...)</li>
- *     <li>{@link String} and array of {@link String}s</li>
- *     <li>{@link UUID} and array of {@link UUID}s</li>
- *     <li>{@link Date} and array of {@link Date}s</li>
- *     <li>{@link Timestamp} and array of {@link Timestamp}s</li>
- *     <li>Enums and array of enums</li>
- *     <li>
- *         Maps, collections and array of objects (but objects inside
- *         them will still be converted if they are portable)
- *     </li>
- * </ul>
- * <h1 class="header">Working With Maps and Collections</h1>
- * All maps and collections in the portable objects are serialized automatically. When working
- * with different platforms, e.g. C++ or .NET, Ignite will automatically pick the most
- * adequate collection or map in either language. For example, {@link ArrayList} in Java will become
- * {@code List} in C#, {@link LinkedList} in Java is {@link LinkedList} in C#, {@link HashMap}
- * in Java is {@code Dictionary} in C#, and {@link TreeMap} in Java becomes {@code SortedDictionary}
- * in C#, etc.
- * <h1 class="header">Building Portable Objects</h1>
- * Ignite comes with {@link PortableBuilder} which allows to build portable objects dynamically:
- * <pre name=code class=java>
- * PortableBuilder builder = Ignition.ignite().portables().builder();
- *
- * builder.typeId("MyObject");
- *
- * builder.stringField("fieldA", "A");
- * build.intField("fieldB", "B");
- *
- * PortableObject portableObj = builder.build();
- * </pre>
- * For the cases when class definition is present
- * in the class path, it is also possible to populate a standard POJO and then
- * convert it to portable format, like so:
- * <pre name=code class=java>
- * MyObject obj = new MyObject();
- *
- * obj.setFieldA("A");
- * obj.setFieldB(123);
- *
- * PortableObject portableObj = Ignition.ignite().portables().toPortable(obj);
- * </pre>
- * NOTE: you don't need to convert typed objects to portable format before storing
- * them in cache, Ignite will do that automatically.
- * <h1 class="header">Portable Metadata</h1>
- * Even though Ignite portable protocol only works with hash codes for type and field names
- * to achieve better performance, Ignite provides metadata for all portable types which
- * can be queried ar runtime via any of the {@link IgnitePortables#metadata(Class)}
- * methods. Having metadata also allows for proper formatting of {@code PortableObject#toString()} method,
- * even when portable objects are kept in binary format only, which may be necessary for audit reasons.
- * <h1 class="header">Dynamic Structure Changes</h1>
- * Since objects are always cached in the portable binary format, server does not need to
- * be aware of the class definitions. Moreover, if class definitions are not present or not
- * used on the server, then clients can continuously change the structure of the portable
- * objects without having to restart the cluster. For example, if one client stores a
- * certain class with fields A and B, and another client stores the same class with
- * fields B and C, then the server-side portable object will have the fields A, B, and C.
- * As the structure of a portable object changes, the new fields become available for SQL queries
- * automatically.
- * <h1 class="header">Configuration</h1>
- * By default all your objects are considered as portables and no specific configuration is needed.
- * However, in some cases, like when an object is used by both Java and .Net, you may need to specify portable objects
- * explicitly by calling {@link PortableMarshaller#setClassNames(Collection)}.
- * The only requirement Ignite imposes is that your object has an empty
- * constructor. Note, that since server side does not have to know the class definition,
- * you only need to list portable objects in configuration on the client side. However, if you
- * list them on the server side as well, then you get the ability to deserialize portable objects
- * into concrete types on the server as well as on the client.
- * <p>
- * Here is an example of portable configuration (note that star (*) notation is supported):
- * <pre name=code class=xml>
- * ...
- * &lt;!-- Explicit portable objects configuration. --&gt;
- * &lt;property name="marshaller"&gt;
- *     &lt;bean class="org.apache.ignite.marshaller.portable.PortableMarshaller"&gt;
- *         &lt;property name="classNames"&gt;
- *             &lt;list&gt;
- *                 &lt;value&gt;my.package.for.portable.objects.*&lt;/value&gt;
- *                 &lt;value&gt;org.apache.ignite.examples.client.portable.Employee&lt;/value&gt;
- *             &lt;/list&gt;
- *         &lt;/property&gt;
- *     &lt;/bean&gt;
- * &lt;/property&gt;
- * ...
- * </pre>
- * or from code:
- * <pre name=code class=java>
- * IgniteConfiguration cfg = new IgniteConfiguration();
- *
- * PortableMarshaller marsh = new PortableMarshaller();
- *
- * marsh.setClassNames(Arrays.asList(
- *     Employee.class.getName(),
- *     Address.class.getName())
- * );
- *
- * cfg.setMarshaller(marsh);
- * </pre>
- * You can also specify class name for a portable object via {@link PortableTypeConfiguration}.
- * Do it in case if you need to override other configuration properties on per-type level, like
- * ID-mapper, or serializer.
- * <h1 class="header">Custom Affinity Keys</h1>
- * Often you need to specify an alternate key (not the cache key) for affinity routing whenever
- * storing objects in cache. For example, if you are caching {@code Employee} object with
- * {@code Organization}, and want to colocate employees with organization they work for,
- * so you can process them together, you need to specify an alternate affinity key.
- * With portable objects you would have to do it as following:
- * <pre name=code class=xml>
- * &lt;property name="marshaller"&gt;
- *     &lt;bean class="org.gridgain.grid.marshaller.portable.PortableMarshaller"&gt;
- *         ...
- *         &lt;property name="typeConfigurations"&gt;
- *             &lt;list&gt;
- *                 &lt;bean class="org.apache.ignite.portable.PortableTypeConfiguration"&gt;
- *                     &lt;property name="className" value="org.apache.ignite.examples.client.portable.EmployeeKey"/&gt;
- *                     &lt;property name="affinityKeyFieldName" value="organizationId"/&gt;
- *                 &lt;/bean&gt;
- *             &lt;/list&gt;
- *         &lt;/property&gt;
- *         ...
- *     &lt;/bean&gt;
- * &lt;/property&gt;
- * </pre>
- * <h1 class="header">Serialization</h1>
- * Serialization and deserialization works out-of-the-box in Ignite. However, you can provide your own custom
- * serialization logic by optionally implementing {@link PortableMarshalAware} interface, like so:
- * <pre name=code class=java>
- * public class Address implements PortableMarshalAware {
- *     private String street;
- *     private int zip;
- *
- *     // Empty constructor required for portable deserialization.
- *     public Address() {}
- *
- *     &#64;Override public void writePortable(PortableWriter writer) throws PortableException {
- *         writer.writeString("street", street);
- *         writer.writeInt("zip", zip);
- *     }
- *
- *     &#64;Override public void readPortable(PortableReader reader) throws PortableException {
- *         street = reader.readString("street");
- *         zip = reader.readInt("zip");
- *     }
- * }
- * </pre>
- * Alternatively, if you cannot change class definitions, you can provide custom serialization
- * logic in {@link PortableSerializer} either globally in {@link PortableMarshaller} or
- * for a specific type via {@link PortableTypeConfiguration} instance.
- * <p>
- * Similar to java serialization you can use {@code writeReplace()} and {@code readResolve()} methods.
- * <ul>
- *     <li>
- *         {@code readResolve} is defined as follows: {@code ANY-ACCESS-MODIFIER Object readResolve()}.
- *         It may be used to replace the de-serialized object by another one of your choice.
- *     </li>
- *     <li>
- *          {@code writeReplace} is defined as follows: {@code ANY-ACCESS-MODIFIER Object writeReplace()}. This method
- *          allows the developer to provide a replacement object that will be serialized instead of the original one.
- *     </li>
- * </ul>
- *
- * <h1 class="header">Custom ID Mappers</h1>
- * Ignite implementation uses name hash codes to generate IDs for class names or field names
- * internally. However, in cases when you want to provide your own ID mapping schema,
- * you can provide your own {@link PortableIdMapper} implementation.
- * <p>
- * ID-mapper may be provided either globally in {@link PortableMarshaller},
- * or for a specific type via {@link PortableTypeConfiguration} instance.
- * <h1 class="header">Query Indexing</h1>
- * Portable objects can be indexed for querying by specifying index fields in
- * {@link org.apache.ignite.cache.CacheTypeMetadata} inside of specific
- * {@link org.apache.ignite.configuration.CacheConfiguration} instance,
- * like so:
- * <pre name=code class=xml>
- * ...
- * &lt;bean class="org.apache.ignite.cache.CacheConfiguration"&gt;
- *     ...
- *     &lt;property name="typeMetadata"&gt;
- *         &lt;list&gt;
- *             &lt;bean class="CacheTypeMetadata"&gt;
- *                 &lt;property name="type" value="Employee"/&gt;
- *
- *                 &lt;!-- Fields to index in ascending order. --&gt;
- *                 &lt;property name="ascendingFields"&gt;
- *                     &lt;map&gt;
- *                     &lt;entry key="name" value="java.lang.String"/&gt;
- *
- *                         &lt;!-- Nested portable objects can also be indexed. --&gt;
- *                         &lt;entry key="address.zip" value="java.lang.Integer"/&gt;
- *                     &lt;/map&gt;
- *                 &lt;/property&gt;
- *             &lt;/bean&gt;
- *         &lt;/list&gt;
- *     &lt;/property&gt;
- * &lt;/bean&gt;
- * </pre>
- */
-public interface IgnitePortables {
-    /**
-     * Gets type ID for given type name.
-     *
-     * @param typeName Type name.
-     * @return Type ID.
-     */
-    public int typeId(String typeName);
-
-    /**
-     * Converts provided object to instance of {@link PortableObject}.
-     *
-     * @param obj Object to convert.
-     * @return Converted object.
-     * @throws PortableException In case of error.
-     */
-    public <T> T toPortable(@Nullable Object obj) throws PortableException;
-
-    /**
-     * Creates new portable builder.
-     *
-     * @param typeId ID of the type.
-     * @return Newly portable builder.
-     */
-    public PortableBuilder builder(int typeId);
-
-    /**
-     * Creates new portable builder.
-     *
-     * @param typeName Type name.
-     * @return Newly portable builder.
-     */
-    public PortableBuilder builder(String typeName);
-
-    /**
-     * Creates portable builder initialized by existing portable object.
-     *
-     * @param portableObj Portable object to initialize builder.
-     * @return Portable builder.
-     */
-    public PortableBuilder builder(PortableObject portableObj);
-
-    /**
-     * Gets metadata for provided class.
-     *
-     * @param cls Class.
-     * @return Metadata.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public PortableMetadata metadata(Class<?> cls) throws PortableException;
-
-    /**
-     * Gets metadata for provided class name.
-     *
-     * @param typeName Type name.
-     * @return Metadata.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public PortableMetadata metadata(String typeName) throws PortableException;
-
-    /**
-     * Gets metadata for provided type ID.
-     *
-     * @param typeId Type ID.
-     * @return Metadata.
-     * @throws PortableException In case of error.
-     */
-    @Nullable public PortableMetadata metadata(int typeId) throws PortableException;
-
-    /**
-     * Gets metadata for all known types.
-     *
-     * @return Metadata.
-     * @throws PortableException In case of error.
-     */
-    public Collection<PortableMetadata> metadata() throws PortableException;
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/binary/BinaryField.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryField.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryField.java
new file mode 100644
index 0000000..35aa191
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryField.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.binary;
+
+/**
+ * Binary object field. Can be used to speed object field lookup.
+ */
+public interface BinaryField {
+    /**
+     * Get field's name.
+     *
+     * @return Name.
+     */
+    public String name();
+
+    /**
+     * Check whether field exists in the object.
+     *
+     * @param obj Object.
+     * @return {@code True} if exists.
+     */
+    public boolean exists(BinaryObject obj);
+
+    /**
+     * Get field's value from the given object.
+     *
+     * @param obj Object.
+     * @return Value.
+     */
+    public <T> T value(BinaryObject obj);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/binary/BinaryInvalidTypeException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryInvalidTypeException.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryInvalidTypeException.java
new file mode 100644
index 0000000..2fe1e79
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryInvalidTypeException.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.binary;
+
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Exception indicating that class needed for deserialization of binary object does not exist.
+ * <p>
+ * Thrown from {@link BinaryObject#deserialize()} method.
+ */
+public class BinaryInvalidTypeException extends BinaryObjectException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * Creates invalid class exception with error message.
+     *
+     * @param msg Error message.
+     */
+    public BinaryInvalidTypeException(String msg) {
+        super(msg);
+    }
+
+    /**
+     * Creates invalid class exception with {@link Throwable} as a cause.
+     *
+     * @param cause Cause.
+     */
+    public BinaryInvalidTypeException(Throwable cause) {
+        super(cause);
+    }
+
+    /**
+     * Creates invalid class exception with error message and {@link Throwable} as a cause.
+     *
+     * @param msg Error message.
+     * @param cause Cause.
+     */
+    public BinaryInvalidTypeException(String msg, @Nullable Throwable cause) {
+        super(msg, cause);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/binary/BinaryObject.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryObject.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryObject.java
new file mode 100644
index 0000000..f48f350
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryObject.java
@@ -0,0 +1,163 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.binary;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.TreeMap;
+import org.apache.ignite.marshaller.portable.PortableMarshaller;
+
+/**
+ * Wrapper for binary object in binary format. Once an object is defined as binary,
+ * Ignite will always store it in memory in the binary format.
+ * User can choose to work either with the binary format or with the deserialized form
+ * (assuming that class definitions are present in the classpath).
+ * <p>
+ * <b>NOTE:</b> user does not need to (and should not) implement this interface directly.
+ * <p>
+ * To work with the binary format directly, user should create a cache projection
+ * over {@code BinaryObject} class and then retrieve individual fields as needed:
+ * <pre name=code class=java>
+ * IgniteCache&lt;BinaryObject, BinaryObject&gt; prj = cache.withKeepBinary();
+ *
+ * // Convert instance of MyKey to binary format.
+ * // We could also use BinaryObjectBuilder to create the key in binary format directly.
+ * BinaryObject key = ignite.binary().toBinary(new MyKey());
+ *
+ * BinaryObject val = prj.get(key);
+ *
+ * String field = val.field("myFieldName");
+ * </pre>
+ * Alternatively, if we have class definitions in the classpath, we may choose to work with deserialized
+ * typed objects at all times. In this case we do incur the deserialization cost. However, if
+ * {@link PortableMarshaller#isKeepDeserialized()} is {@code true} then Ignite will only deserialize on the first access
+ * and will cache the deserialized object, so it does not have to be deserialized again:
+ * <pre name=code class=java>
+ * IgniteCache&lt;MyKey.class, MyValue.class&gt; cache = grid.cache(null);
+ *
+ * MyValue val = cache.get(new MyKey());
+ *
+ * // Normal java getter.
+ * String fieldVal = val.getMyFieldName();
+ * </pre>
+ * <h1 class="header">Working With Maps and Collections</h1>
+ * All maps and collections in binary objects are serialized automatically. When working
+ * with different platforms, e.g. C++ or .NET, Ignite will automatically pick the most
+ * adequate collection or map in either language. For example, {@link ArrayList} in Java will become
+ * {@code List} in C#, {@link LinkedList} in Java is {@link LinkedList} in C#, {@link HashMap}
+ * in Java is {@code Dictionary} in C#, and {@link TreeMap} in Java becomes {@code SortedDictionary}
+ * in C#, etc.
+ * <h1 class="header">Dynamic Structure Changes</h1>
+ * Since objects are always cached in the binary format, server does not need to
+ * be aware of the class definitions. Moreover, if class definitions are not present or not
+ * used on the server, then clients can continuously change the structure of the binary
+ * objects without having to restart the cluster. For example, if one client stores a
+ * certain class with fields A and B, and another client stores the same class with
+ * fields B and C, then the server-side binary object will have the fields A, B, and C.
+ * As the structure of a binary object changes, the new fields become available for SQL queries
+ * automatically.
+ * <h1 class="header">Building Binary Objects</h1>
+ * Ignite comes with {@link BinaryObjectBuilder} which allows to build binary objects dynamically:
+ * <pre name=code class=java>
+ * BinaryObjectBuilder builder = Ignition.ignite().binary().builder("org.project.MyObject");
+ *
+ * builder.setField("fieldA", "A");
+ * builder.setField("fieldB", "B");
+ *
+ * BinaryObject binaryObj = builder.build();
+ * </pre>
+ * For the cases when class definition is present
+ * in the class path, it is also possible to populate a standard POJO and then
+ * convert it to binary format, like so:
+ * <pre name=code class=java>
+ * MyObject obj = new MyObject();
+ *
+ * obj.setFieldA("A");
+ * obj.setFieldB(123);
+ *
+ * BinaryObject binaryObj = Ignition.ignite().binary().toBinary(obj);
+ * </pre>
+ * <h1 class="header">Binary Type Metadata</h1>
+ * Even though Ignite binary protocol only works with hash codes for type and field names
+ * to achieve better performance, Ignite provides metadata for all binary types which
+ * can be queried ar runtime via any of the {@link org.apache.ignite.IgniteBinary#metadata(Class)}
+ * methods. Having metadata also allows for proper formatting of {@code BinaryObject.toString()} method,
+ * even when binary objects are kept in binary format only, which may be necessary for audit reasons.
+ */
+public interface BinaryObject extends Serializable, Cloneable {
+    /**
+     * Gets binary object type ID.
+     *
+     * @return Type ID.
+     */
+    public int typeId();
+
+    /**
+     * Gets type information for this binary object.
+     *
+     * @return Binary object type information.
+     * @throws BinaryObjectException In case of error.
+     */
+    public BinaryType type() throws BinaryObjectException;
+
+    /**
+     * Gets field value.
+     *
+     * @param fieldName Field name.
+     * @return Field value.
+     * @throws BinaryObjectException In case of any other error.
+     * TODO ignite-1282 remove.
+     */
+    public <F> F field(String fieldName) throws BinaryObjectException;
+
+    /**
+     * Checks whether field is set.
+     ** TODO ignite-1282 remove.
+     *
+     * @param fieldName Field name.
+     * @return {@code true} if field is set.
+     */
+    public boolean hasField(String fieldName);
+
+    /**
+     * Gets field descriptor.
+     *
+     * @param fieldName Field name.
+     * @return Field descriptor.
+     * @throws BinaryObjectException If failed.
+     */
+    public BinaryField fieldDescriptor(String fieldName) throws BinaryObjectException;
+
+    /**
+     * Gets fully deserialized instance of binary object.
+     *
+     * @return Fully deserialized instance of binary object.
+     * @throws BinaryInvalidTypeException If class doesn't exist.
+     * @throws BinaryObjectException In case of any other error.
+     */
+    public <T> T deserialize() throws BinaryObjectException;
+
+    /**
+     * Copies this binary object.
+     *
+     * @return Copy of this binary object.
+     */
+    public BinaryObject clone() throws CloneNotSupportedException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/binary/BinaryObjectBuilder.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryObjectBuilder.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryObjectBuilder.java
new file mode 100644
index 0000000..507b0fc
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryObjectBuilder.java
@@ -0,0 +1,136 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.binary;
+
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Binary object builder. Provides ability to build binary objects dynamically without having class definitions.
+ * <p>
+ * Here is an example of how a binary object can be built dynamically:
+ * <pre name=code class=java>
+ * BinaryObjectBuilder builder = Ignition.ignite().binary().builder("org.project.MyObject");
+ *
+ * builder.setField("fieldA", "A");
+ * builder.setField("fieldB", "B");
+ *
+ * BinaryObject binaryObj = builder.build();
+ * </pre>
+ *
+ * <p>
+ * Also builder can be initialized by existing binary object. This allows changing some fields without affecting
+ * other fields.
+ * <pre name=code class=java>
+ * BinaryObjectBuilder builder = Ignition.ignite().binary().builder(person);
+ *
+ * builder.setField("name", "John");
+ *
+ * person = builder.build();
+ * </pre>
+ * </p>
+ *
+ * If you need to modify nested binary object you can get builder for nested object using
+ * {@link #getField(String)}, changes made on nested builder will affect parent object,
+ * for example:
+ *
+ * <pre name=code class=java>
+ * BinaryObjectBuilder personBuilder = grid.binary().createBuilder(personBinaryObj);
+ * BinaryObjectBuilder addressBuilder = personBuilder.setField("address");
+ *
+ * addressBuilder.setField("city", "New York");
+ *
+ * personBinaryObj = personBuilder.build();
+ *
+ * // Should be "New York".
+ * String city = personBinaryObj.getField("address").getField("city");
+ * </pre>
+ *
+ * @see org.apache.ignite.IgniteBinary#builder(int)
+ * @see org.apache.ignite.IgniteBinary#builder(String)
+ * @see org.apache.ignite.IgniteBinary#builder(BinaryObject)
+ */
+public interface BinaryObjectBuilder {
+    /**
+     * Returns value assigned to the specified field.
+     * If the value is a binary object instance of {@code BinaryObjectBuilder} will be returned,
+     * which can be modified.
+     * <p>
+     * Collections and maps returned from this method are modifiable.
+     *
+     * @param name Field name.
+     * @return Filed value.
+     */
+    public <T> T getField(String name);
+
+    /**
+     * Sets field value.
+     *
+     * @param name Field name.
+     * @param val Field value (cannot be {@code null}).
+     * @see BinaryObject#type()
+     */
+    public BinaryObjectBuilder setField(String name, Object val);
+
+    /**
+     * Sets field value with value type specification.
+     * <p>
+     * Field type is needed for proper metadata update.
+     *
+     * @param name Field name.
+     * @param val Field value.
+     * @param type Field type.
+     * @see BinaryObject#type()
+     */
+    public <T> BinaryObjectBuilder setField(String name, @Nullable T val, Class<? super T> type);
+
+    /**
+     * Sets field value.
+     * <p>
+     * This method should be used if field is binary object.
+     *
+     * @param name Field name.
+     * @param builder Builder for object field.
+     */
+    public BinaryObjectBuilder setField(String name, @Nullable BinaryObjectBuilder builder);
+
+    /**
+     * Removes field from this builder.
+     *
+     * @param fieldName Field name.
+     * @return {@code this} instance for chaining.
+     */
+    public BinaryObjectBuilder removeField(String fieldName);
+
+    /**
+     * Sets hash code for resulting binary object returned by {@link #build()} method.
+     * <p>
+     * If not set {@code 0} is used.
+     *
+     * @param hashCode Hash code.
+     * @return {@code this} instance for chaining.
+     */
+    public BinaryObjectBuilder hashCode(int hashCode);
+
+    /**
+     * Builds binary object.
+     *
+     * @return Binary object.
+     * @throws BinaryObjectException In case of error.
+     */
+    public BinaryObject build() throws BinaryObjectException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/binary/BinaryObjectException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryObjectException.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryObjectException.java
new file mode 100644
index 0000000..4305382
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryObjectException.java
@@ -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.
+ */
+
+package org.apache.ignite.binary;
+
+import org.apache.ignite.IgniteException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Exception indicating binary object serialization error.
+ */
+public class BinaryObjectException extends IgniteException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * Creates binary exception with error message.
+     *
+     * @param msg Error message.
+     */
+    public BinaryObjectException(String msg) {
+        super(msg);
+    }
+
+    /**
+     * Creates binary exception with {@link Throwable} as a cause.
+     *
+     * @param cause Cause.
+     */
+    public BinaryObjectException(Throwable cause) {
+        super(cause);
+    }
+
+    /**
+     * Creates binary exception with error message and {@link Throwable} as a cause.
+     *
+     * @param msg Error message.
+     * @param cause Cause.
+     */
+    public BinaryObjectException(String msg, @Nullable Throwable cause) {
+        super(msg, cause);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/binary/BinaryRawReader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryRawReader.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryRawReader.java
new file mode 100644
index 0000000..7ff515a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryRawReader.java
@@ -0,0 +1,240 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.binary;
+
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.util.Collection;
+import java.util.Date;
+import java.util.Map;
+import java.util.UUID;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Raw reader for binary objects. Raw reader does not use field name hash codes, therefore,
+ * making the format even more compact. However, if the raw reader is used,
+ * dynamic structure changes to the binary objects are not supported.
+ */
+public interface BinaryRawReader {
+    /**
+     * @return Byte value.
+     * @throws BinaryObjectException In case of error.
+     */
+    public byte readByte() throws BinaryObjectException;
+
+    /**
+     * @return Short value.
+     * @throws BinaryObjectException In case of error.
+     */
+    public short readShort() throws BinaryObjectException;
+
+    /**
+     * @return Integer value.
+     * @throws BinaryObjectException In case of error.
+     */
+    public int readInt() throws BinaryObjectException;
+
+    /**
+     * @return Long value.
+     * @throws BinaryObjectException In case of error.
+     */
+    public long readLong() throws BinaryObjectException;
+
+    /**
+     * @return Float value.
+     * @throws BinaryObjectException In case of error.
+     */
+    public float readFloat() throws BinaryObjectException;
+
+    /**
+     * @return Double value.
+     * @throws BinaryObjectException In case of error.
+     */
+    public double readDouble() throws BinaryObjectException;
+
+    /**
+     * @return Char value.
+     * @throws BinaryObjectException In case of error.
+     */
+    public char readChar() throws BinaryObjectException;
+
+    /**
+     * @return Boolean value.
+     * @throws BinaryObjectException In case of error.
+     */
+    public boolean readBoolean() throws BinaryObjectException;
+
+    /**
+     * @return Decimal value.
+     * @throws BinaryObjectException In case of error.
+     */
+    @Nullable public BigDecimal readDecimal() throws BinaryObjectException;
+
+    /**
+     * @return String value.
+     * @throws BinaryObjectException In case of error.
+     */
+    @Nullable public String readString() throws BinaryObjectException;
+
+    /**
+     * @return UUID.
+     * @throws BinaryObjectException In case of error.
+     */
+    @Nullable public UUID readUuid() throws BinaryObjectException;
+
+    /**
+     * @return Date.
+     * @throws BinaryObjectException In case of error.
+     */
+    @Nullable public Date readDate() throws BinaryObjectException;
+
+    /**
+     * @return Timestamp.
+     * @throws BinaryObjectException In case of error.
+     */
+    @Nullable public Timestamp readTimestamp() throws BinaryObjectException;
+
+    /**
+     * @return Object.
+     * @throws BinaryObjectException In case of error.
+     */
+    @Nullable public <T> T readObject() throws BinaryObjectException;
+
+    /**
+     * @return Byte array.
+     * @throws BinaryObjectException In case of error.
+     */
+    @Nullable public byte[] readByteArray() throws BinaryObjectException;
+
+    /**
+     * @return Short array.
+     * @throws BinaryObjectException In case of error.
+     */
+    @Nullable public short[] readShortArray() throws BinaryObjectException;
+
+    /**
+     * @return Integer array.
+     * @throws BinaryObjectException In case of error.
+     */
+    @Nullable public int[] readIntArray() throws BinaryObjectException;
+
+    /**
+     * @return Long array.
+     * @throws BinaryObjectException In case of error.
+     */
+    @Nullable public long[] readLongArray() throws BinaryObjectException;
+
+    /**
+     * @return Float array.
+     * @throws BinaryObjectException In case of error.
+     */
+    @Nullable public float[] readFloatArray() throws BinaryObjectException;
+
+    /**
+     * @return Byte array.
+     * @throws BinaryObjectException In case of error.
+     */
+    @Nullable public double[] readDoubleArray() throws BinaryObjectException;
+
+    /**
+     * @return Char array.
+     * @throws BinaryObjectException In case of error.
+     */
+    @Nullable public char[] readCharArray() throws BinaryObjectException;
+
+    /**
+     * @return Boolean array.
+     * @throws BinaryObjectException In case of error.
+     */
+    @Nullable public boolean[] readBooleanArray() throws BinaryObjectException;
+
+    /**
+     * @return Decimal array.
+     * @throws BinaryObjectException In case of error.
+     */
+    @Nullable public BigDecimal[] readDecimalArray() throws BinaryObjectException;
+
+    /**
+     * @return String array.
+     * @throws BinaryObjectException In case of error.
+     */
+    @Nullable public String[] readStringArray() throws BinaryObjectException;
+
+    /**
+     * @return UUID array.
+     * @throws BinaryObjectException In case of error.
+     */
+    @Nullable public UUID[] readUuidArray() throws BinaryObjectException;
+
+    /**
+     * @return Date array.
+     * @throws BinaryObjectException In case of error.
+     */
+    @Nullable public Date[] readDateArray() throws BinaryObjectException;
+
+    /**
+     * @return Timestamp array.
+     * @throws BinaryObjectException In case of error.
+     */
+    @Nullable public Timestamp[] readTimestampArray() throws BinaryObjectException;
+
+    /**
+     * @return Object array.
+     * @throws BinaryObjectException In case of error.
+     */
+    @Nullable public Object[] readObjectArray() throws BinaryObjectException;
+
+    /**
+     * @return Collection.
+     * @throws BinaryObjectException In case of error.
+     */
+    @Nullable public <T> Collection<T> readCollection() throws BinaryObjectException;
+
+    /**
+     * @param colCls Collection class.
+     * @return Collection.
+     * @throws BinaryObjectException In case of error.
+     */
+    @Nullable public <T> Collection<T> readCollection(Class<? extends Collection<T>> colCls)
+        throws BinaryObjectException;
+
+    /**
+     * @return Map.
+     * @throws BinaryObjectException In case of error.
+     */
+    @Nullable public <K, V> Map<K, V> readMap() throws BinaryObjectException;
+
+    /**
+     * @param mapCls Map class.
+     * @return Map.
+     * @throws BinaryObjectException In case of error.
+     */
+    @Nullable public <K, V> Map<K, V> readMap(Class<? extends Map<K, V>> mapCls) throws BinaryObjectException;
+
+    /**
+     * @return Value.
+     * @throws BinaryObjectException In case of error.
+     */
+    @Nullable public <T extends Enum<?>> T readEnum() throws BinaryObjectException;
+
+    /**
+     * @return Value.
+     * @throws BinaryObjectException In case of error.
+     */
+    @Nullable public <T extends Enum<?>> T[] readEnumArray() throws BinaryObjectException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/binary/BinaryRawWriter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryRawWriter.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryRawWriter.java
new file mode 100644
index 0000000..f283c06
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryRawWriter.java
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.binary;
+
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.util.Collection;
+import java.util.Date;
+import java.util.Map;
+import java.util.UUID;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Raw writer for binary object. Raw writer does not write field name hash codes, therefore,
+ * making the format even more compact. However, if the raw writer is used,
+ * dynamic structure changes to the binary objects are not supported.
+ */
+public interface BinaryRawWriter {
+    /**
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeByte(byte val) throws BinaryObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeShort(short val) throws BinaryObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeInt(int val) throws BinaryObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeLong(long val) throws BinaryObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeFloat(float val) throws BinaryObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeDouble(double val) throws BinaryObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeChar(char val) throws BinaryObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeBoolean(boolean val) throws BinaryObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeDecimal(@Nullable BigDecimal val) throws BinaryObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeString(@Nullable String val) throws BinaryObjectException;
+
+    /**
+     * @param val UUID to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeUuid(@Nullable UUID val) throws BinaryObjectException;
+
+    /**
+     * @param val Date to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeDate(@Nullable Date val) throws BinaryObjectException;
+
+    /**
+     * @param val Timestamp to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeTimestamp(@Nullable Timestamp val) throws BinaryObjectException;
+
+    /**
+     * @param obj Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeObject(@Nullable Object obj) throws BinaryObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeByteArray(@Nullable byte[] val) throws BinaryObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeShortArray(@Nullable short[] val) throws BinaryObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeIntArray(@Nullable int[] val) throws BinaryObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeLongArray(@Nullable long[] val) throws BinaryObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeFloatArray(@Nullable float[] val) throws BinaryObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeDoubleArray(@Nullable double[] val) throws BinaryObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeCharArray(@Nullable char[] val) throws BinaryObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeBooleanArray(@Nullable boolean[] val) throws BinaryObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeDecimalArray(@Nullable BigDecimal[] val) throws BinaryObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeStringArray(@Nullable String[] val) throws BinaryObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeUuidArray(@Nullable UUID[] val) throws BinaryObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeDateArray(@Nullable Date[] val) throws BinaryObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeTimestampArray(@Nullable Timestamp[] val) throws BinaryObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeObjectArray(@Nullable Object[] val) throws BinaryObjectException;
+
+    /**
+     * @param col Collection to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public <T> void writeCollection(@Nullable Collection<T> col) throws BinaryObjectException;
+
+    /**
+     * @param map Map to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public <K, V> void writeMap(@Nullable Map<K, V> map) throws BinaryObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public <T extends Enum<?>> void writeEnum(T val) throws BinaryObjectException;
+
+    /**
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public <T extends Enum<?>> void writeEnumArray(T[] val) throws BinaryObjectException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/binary/BinaryReader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryReader.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryReader.java
new file mode 100644
index 0000000..8a89a87
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryReader.java
@@ -0,0 +1,290 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.binary;
+
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.util.Collection;
+import java.util.Date;
+import java.util.Map;
+import java.util.UUID;
+
+/**
+ * Reader for portable objects used in {@link Binarylizable} implementations.
+ * Useful for the cases when user wants a fine-grained control over serialization.
+ * <p>
+ * Note that Ignite never writes full strings for field or type names. Instead,
+ * for performance reasons, Ignite writes integer hash codes for type and field names.
+ * It has been tested that hash code conflicts for the type names or the field names
+ * within the same type are virtually non-existent and, to gain performance, it is safe
+ * to work with hash codes. For the cases when hash codes for different types or fields
+ * actually do collide, Ignite provides {@link BinaryTypeIdMapper} which
+ * allows to override the automatically generated hash code IDs for the type and field names.
+ */
+public interface BinaryReader {
+    /**
+     * @param fieldName Field name.
+     * @return Byte value.
+     * @throws BinaryObjectException In case of error.
+     */
+    public byte readByte(String fieldName) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Short value.
+     * @throws BinaryObjectException In case of error.
+     */
+    public short readShort(String fieldName) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Integer value.
+     * @throws BinaryObjectException In case of error.
+     */
+    public int readInt(String fieldName) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Long value.
+     * @throws BinaryObjectException In case of error.
+     */
+    public long readLong(String fieldName) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @throws BinaryObjectException In case of error.
+     * @return Float value.
+     */
+    public float readFloat(String fieldName) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Double value.
+     * @throws BinaryObjectException In case of error.
+     */
+    public double readDouble(String fieldName) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Char value.
+     * @throws BinaryObjectException In case of error.
+     */
+    public char readChar(String fieldName) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Boolean value.
+     * @throws BinaryObjectException In case of error.
+     */
+    public boolean readBoolean(String fieldName) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Decimal value.
+     * @throws BinaryObjectException In case of error.
+     */
+    public BigDecimal readDecimal(String fieldName) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return String value.
+     * @throws BinaryObjectException In case of error.
+     */
+    public String readString(String fieldName) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return UUID.
+     * @throws BinaryObjectException In case of error.
+     */
+    public UUID readUuid(String fieldName) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Date.
+     * @throws BinaryObjectException In case of error.
+     */
+    public Date readDate(String fieldName) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Timestamp.
+     * @throws BinaryObjectException In case of error.
+     */
+    public Timestamp readTimestamp(String fieldName) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Object.
+     * @throws BinaryObjectException In case of error.
+     */
+    public <T> T readObject(String fieldName) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Byte array.
+     * @throws BinaryObjectException In case of error.
+     */
+    public byte[] readByteArray(String fieldName) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Short array.
+     * @throws BinaryObjectException In case of error.
+     */
+    public short[] readShortArray(String fieldName) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Integer array.
+     * @throws BinaryObjectException In case of error.
+     */
+    public int[] readIntArray(String fieldName) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Long array.
+     * @throws BinaryObjectException In case of error.
+     */
+    public long[] readLongArray(String fieldName) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Float array.
+     * @throws BinaryObjectException In case of error.
+     */
+    public float[] readFloatArray(String fieldName) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Byte array.
+     * @throws BinaryObjectException In case of error.
+     */
+    public double[] readDoubleArray(String fieldName) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Char array.
+     * @throws BinaryObjectException In case of error.
+     */
+    public char[] readCharArray(String fieldName) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Boolean array.
+     * @throws BinaryObjectException In case of error.
+     */
+    public boolean[] readBooleanArray(String fieldName) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Decimal array.
+     * @throws BinaryObjectException In case of error.
+     */
+    public BigDecimal[] readDecimalArray(String fieldName) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return String array.
+     * @throws BinaryObjectException In case of error.
+     */
+    public String[] readStringArray(String fieldName) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return UUID array.
+     * @throws BinaryObjectException In case of error.
+     */
+    public UUID[] readUuidArray(String fieldName) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Date array.
+     * @throws BinaryObjectException In case of error.
+     */
+    public Date[] readDateArray(String fieldName) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Timestamp array.
+     * @throws BinaryObjectException In case of error.
+     */
+    public Timestamp[] readTimestampArray(String fieldName) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Object array.
+     * @throws BinaryObjectException In case of error.
+     */
+    public Object[] readObjectArray(String fieldName) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Collection.
+     * @throws BinaryObjectException In case of error.
+     */
+    public <T> Collection<T> readCollection(String fieldName) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param colCls Collection class.
+     * @return Collection.
+     * @throws BinaryObjectException In case of error.
+     */
+    public <T> Collection<T> readCollection(String fieldName, Class<? extends Collection<T>> colCls)
+        throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Map.
+     * @throws BinaryObjectException In case of error.
+     */
+    public <K, V> Map<K, V> readMap(String fieldName) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param mapCls Map class.
+     * @return Map.
+     * @throws BinaryObjectException In case of error.
+     */
+    public <K, V> Map<K, V> readMap(String fieldName, Class<? extends Map<K, V>> mapCls)
+        throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Value.
+     * @throws BinaryObjectException In case of error.
+     */
+    public <T extends Enum<?>> T readEnum(String fieldName) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Value.
+     * @throws BinaryObjectException In case of error.
+     */
+    public <T extends Enum<?>> T[] readEnumArray(String fieldName) throws BinaryObjectException;
+
+    /**
+     * Gets raw reader. Raw reader does not use field name hash codes, therefore,
+     * making the format even more compact. However, if the raw reader is used,
+     * dynamic structure changes to the binary objects are not supported.
+     *
+     * @return Raw reader.
+     */
+    public BinaryRawReader rawReader();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/binary/BinarySerializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinarySerializer.java b/modules/core/src/main/java/org/apache/ignite/binary/BinarySerializer.java
new file mode 100644
index 0000000..6fa4237
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinarySerializer.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.binary;
+
+import org.apache.ignite.marshaller.portable.PortableMarshaller;
+
+/**
+ * Interface that allows to implement custom serialization logic for binary objects.
+ * Can be used instead of {@link Binarylizable} in case if the class
+ * cannot be changed directly.
+ * <p>
+ * Binary serializer can be configured for all binary objects via
+ * {@link PortableMarshaller#getSerializer()} method, or for a specific
+ * binary type via {@link BinaryTypeConfiguration#getSerializer()} method.
+ */
+public interface BinarySerializer {
+    /**
+     * Writes fields to provided writer.
+     *
+     * @param obj Empty object.
+     * @param writer Binary object writer.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeBinary(Object obj, BinaryWriter writer) throws BinaryObjectException;
+
+    /**
+     * Reads fields from provided reader.
+     *
+     * @param obj Empty object
+     * @param reader Binary object reader.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void readBinary(Object obj, BinaryReader reader) throws BinaryObjectException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/binary/BinaryType.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryType.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryType.java
new file mode 100644
index 0000000..d149fd4
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryType.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.binary;
+
+import java.util.Collection;
+
+/**
+ * Binary type meta data. Metadata for binary types can be accessed from any of the
+ * {@link org.apache.ignite.IgniteBinary#metadata(String)} methods.
+ * Having metadata also allows for proper formatting of {@code BinaryObject#toString()} method,
+ * even when binary objects are kept in binary format only, which may be necessary for audit reasons.
+ */
+public interface BinaryType {
+    /**
+     * Gets binary type name.
+     *
+     * @return Binary type name.
+     */
+    public String typeName();
+
+    /**
+     * Gets collection of all field names for this binary type.
+     *
+     * @return Collection of all field names for this binary type.
+     */
+    public Collection<String> fields();
+
+    /**
+     * Gets name of the field type for a given field.
+     *
+     * @param fieldName Field name.
+     * @return Field type name.
+     */
+    public String fieldTypeName(String fieldName);
+
+    /**
+     * Binary objects can optionally specify custom key-affinity mapping in the
+     * configuration. This method returns the name of the field which should be
+     * used for the key-affinity mapping.
+     *
+     * @return Affinity key field name.
+     */
+    public String affinityKeyFieldName();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/binary/BinaryTypeConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryTypeConfiguration.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryTypeConfiguration.java
new file mode 100644
index 0000000..3b73edb
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryTypeConfiguration.java
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.binary;
+
+import java.util.Collection;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.marshaller.portable.PortableMarshaller;
+
+/**
+ * Defines configuration properties for a specific binary type. Providing per-type
+ * configuration is optional, as it is generally enough, and also optional, to provide global binary
+ * configuration using {@link PortableMarshaller#setClassNames(Collection)}.
+ * However, this class allows you to change configuration properties for a specific
+ * binary type without affecting configuration for other binary types.
+ * <p>
+ * Per-type binary configuration can be specified in {@link PortableMarshaller#getTypeConfigurations()} method.
+ */
+public class BinaryTypeConfiguration {
+    /** Class name. */
+    private String clsName;
+
+    /** ID mapper. */
+    private BinaryTypeIdMapper idMapper;
+
+    /** Serializer. */
+    private BinarySerializer serializer;
+
+    /** Meta data enabled flag. */
+    // TODO ignite-1282.
+    private Boolean metaDataEnabled;
+
+    /** Keep deserialized flag. */
+    private Boolean keepDeserialized;
+
+    /**
+     */
+    public BinaryTypeConfiguration() {
+        // No-op.
+    }
+
+    /**
+     * @param clsName Class name.
+     */
+    public BinaryTypeConfiguration(String clsName) {
+        this.clsName = clsName;
+    }
+
+    /**
+     * Gets type name.
+     *
+     * @return Type name.
+     */
+    public String getClassName() {
+        return clsName;
+    }
+
+    /**
+     * Sets type name.
+     *
+     * @param clsName Type name.
+     */
+    public void setClassName(String clsName) {
+        this.clsName = clsName;
+    }
+
+    /**
+     * Gets ID mapper.
+     *
+     * @return ID mapper.
+     */
+    public BinaryTypeIdMapper getIdMapper() {
+        return idMapper;
+    }
+
+    /**
+     * Sets ID mapper.
+     *
+     * @param idMapper ID mapper.
+     */
+    public void setIdMapper(BinaryTypeIdMapper idMapper) {
+        this.idMapper = idMapper;
+    }
+
+    /**
+     * Gets serializer.
+     *
+     * @return Serializer.
+     */
+    public BinarySerializer getSerializer() {
+        return serializer;
+    }
+
+    /**
+     * Sets serializer.
+     *
+     * @param serializer Serializer.
+     */
+    public void setSerializer(BinarySerializer serializer) {
+        this.serializer = serializer;
+    }
+
+    /**
+     * Defines whether meta data is collected for this type. If provided, this value will override
+     * {@link PortableMarshaller#isMetaDataEnabled()} property.
+     *
+     * @return Whether meta data is collected.
+     */
+    public Boolean isMetaDataEnabled() {
+        return metaDataEnabled;
+    }
+
+    /**
+     * @param metaDataEnabled Whether meta data is collected.
+     */
+    public void setMetaDataEnabled(Boolean metaDataEnabled) {
+        this.metaDataEnabled = metaDataEnabled;
+    }
+
+    /**
+     * Defines whether {@link BinaryObject} should cache deserialized instance. If provided,
+     * this value will override {@link PortableMarshaller#isKeepDeserialized()}
+     * property.
+     *
+     * @return Whether deserialized value is kept.
+     */
+    public Boolean isKeepDeserialized() {
+        return keepDeserialized;
+    }
+
+    /**
+     * @param keepDeserialized Whether deserialized value is kept.
+     */
+    public void setKeepDeserialized(Boolean keepDeserialized) {
+        this.keepDeserialized = keepDeserialized;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(BinaryTypeConfiguration.class, this, super.toString());
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/binary/BinaryTypeIdMapper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryTypeIdMapper.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryTypeIdMapper.java
new file mode 100644
index 0000000..4825f8c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryTypeIdMapper.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.binary;
+
+import org.apache.ignite.marshaller.portable.PortableMarshaller;
+
+/**
+ * Type and field ID mapper for binary objects. Ignite never writes full
+ * strings for field or type names. Instead, for performance reasons, Ignite
+ * writes integer hash codes for type and field names. It has been tested that
+ * hash code conflicts for the type names or the field names
+ * within the same type are virtually non-existent and, to gain performance, it is safe
+ * to work with hash codes. For the cases when hash codes for different types or fields
+ * actually do collide {@code BinaryTypeIdMapper} allows to override the automatically
+ * generated hash code IDs for the type and field names.
+ * <p>
+ * Binary ID mapper can be configured for all binary objects via {@link PortableMarshaller#getIdMapper()} method,
+ * or for a specific binary type via {@link BinaryTypeConfiguration#getIdMapper()} method.
+ */
+public interface BinaryTypeIdMapper {
+    /**
+     * Gets type ID for provided class name.
+     * <p>
+     * If {@code 0} is returned, hash code of class simple name will be used.
+     *
+     * @param clsName Class name.
+     * @return Type ID.
+     */
+    public int typeId(String clsName);
+
+    /**
+     * Gets ID for provided field.
+     * <p>
+     * If {@code 0} is returned, hash code of field name will be used.
+     *
+     * @param typeId Type ID.
+     * @param fieldName Field name.
+     * @return Field ID.
+     */
+    public int fieldId(int typeId, String fieldName);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b783d2b7/modules/core/src/main/java/org/apache/ignite/binary/BinaryWriter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/binary/BinaryWriter.java b/modules/core/src/main/java/org/apache/ignite/binary/BinaryWriter.java
new file mode 100644
index 0000000..0ea4416
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/binary/BinaryWriter.java
@@ -0,0 +1,273 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.binary;
+
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.util.Collection;
+import java.util.Date;
+import java.util.Map;
+import java.util.UUID;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Writer for binary object used in {@link Binarylizable} implementations.
+ * Useful for the cases when user wants a fine-grained control over serialization.
+ * <p>
+ * Note that Ignite never writes full strings for field or type names. Instead,
+ * for performance reasons, Ignite writes integer hash codes for type and field names.
+ * It has been tested that hash code conflicts for the type names or the field names
+ * within the same type are virtually non-existent and, to gain performance, it is safe
+ * to work with hash codes. For the cases when hash codes for different types or fields
+ * actually do collide, Ignite provides {@link BinaryTypeIdMapper} which
+ * allows to override the automatically generated hash code IDs for the type and field names.
+ */
+public interface BinaryWriter {
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeByte(String fieldName, byte val) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeShort(String fieldName, short val) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeInt(String fieldName, int val) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeLong(String fieldName, long val) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeFloat(String fieldName, float val) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeDouble(String fieldName, double val) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeChar(String fieldName, char val) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeBoolean(String fieldName, boolean val) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeDecimal(String fieldName, @Nullable BigDecimal val) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeString(String fieldName, @Nullable String val) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val UUID to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeUuid(String fieldName, @Nullable UUID val) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Date to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeDate(String fieldName, @Nullable Date val) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Timestamp to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeTimestamp(String fieldName, @Nullable Timestamp val) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param obj Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeObject(String fieldName, @Nullable Object obj) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeByteArray(String fieldName, @Nullable byte[] val) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeShortArray(String fieldName, @Nullable short[] val) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeIntArray(String fieldName, @Nullable int[] val) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeLongArray(String fieldName, @Nullable long[] val) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeFloatArray(String fieldName, @Nullable float[] val) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeDoubleArray(String fieldName, @Nullable double[] val) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeCharArray(String fieldName, @Nullable char[] val) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeBooleanArray(String fieldName, @Nullable boolean[] val) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeDecimalArray(String fieldName, @Nullable BigDecimal[] val) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeStringArray(String fieldName, @Nullable String[] val) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeUuidArray(String fieldName, @Nullable UUID[] val) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeDateArray(String fieldName, @Nullable Date[] val) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeTimestampArray(String fieldName, @Nullable Timestamp[] val) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public void writeObjectArray(String fieldName, @Nullable Object[] val) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param col Collection to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public <T> void writeCollection(String fieldName, @Nullable Collection<T> col) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param map Map to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public <K, V> void writeMap(String fieldName, @Nullable Map<K, V> map) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public <T extends Enum<?>> void writeEnum(String fieldName, T val) throws BinaryObjectException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws BinaryObjectException In case of error.
+     */
+    public <T extends Enum<?>> void writeEnumArray(String fieldName, T[] val) throws BinaryObjectException;
+
+    /**
+     * Gets raw writer. Raw writer does not write field name hash codes, therefore,
+     * making the format even more compact. However, if the raw writer is used,
+     * dynamic structure changes to the binary objects are not supported.
+     *
+     * @return Raw writer.
+     */
+    public BinaryRawWriter rawWriter();
+}
\ No newline at end of file