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

[ignite] branch master updated: IGNITE-16038 Java Thin: Retrieve binary configuration from server (#10076)

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

ptupitsyn pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/master by this push:
     new afd4884bbea IGNITE-16038 Java Thin: Retrieve binary configuration from server (#10076)
afd4884bbea is described below

commit afd4884bbea9600e2fa1f15e449857e534c60860
Author: Pavel Tupitsyn <pt...@apache.org>
AuthorDate: Wed Jun 15 09:53:02 2022 +0300

    IGNITE-16038 Java Thin: Retrieve binary configuration from server (#10076)
    
    * Retrieve name mapper and compact footer settings from the cluster and apply them to the client configuration automatically.
    * New behavior can be disabled with `ClientConfiguration#autoBinaryConfigurationEnabled` to support legacy use cases.
    
    https://cwiki.apache.org/confluence/display/IGNITE/IEP-67%3A+Thin+Client+Automatic+Binary+Configuration
---
 .../ignite/configuration/ClientConfiguration.java  |  42 ++++-
 .../internal/client/thin/BinaryNameMapperMode.java |  54 +++++++
 .../client/thin/ClientBinaryMarshaller.java        |   4 +-
 .../client/thin/ClientChannelConfiguration.java    |  11 ++
 .../thin/ClientInternalBinaryConfiguration.java    |  55 +++++++
 .../internal/client/thin/ClientOperation.java      |  11 +-
 .../client/thin/ProtocolBitmaskFeature.java        |   3 +
 .../internal/client/thin/ReliableChannel.java      |   2 +-
 .../internal/client/thin/TcpIgniteClient.java      |  49 ++++++
 .../ignite/client/BinaryConfigurationTest.java     | 175 +++++++++++++++++++++
 .../ignite/client/ClientConfigurationTest.java     |   1 +
 .../org/apache/ignite/client/ReliabilityTest.java  |   2 +-
 .../client/thin/AbstractThinClientTest.java        |  38 ++++-
 .../org/apache/ignite/client/ClientTestSuite.java  |   3 +-
 14 files changed, 430 insertions(+), 20 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/ClientConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/ClientConfiguration.java
index b9ce2195dc9..7a0ffbefd36 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/ClientConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/ClientConfiguration.java
@@ -41,7 +41,7 @@ public final class ClientConfiguration implements Serializable {
     private static final long serialVersionUID = 0L;
 
     /** @serial Server addresses. */
-    private String[] addrs = null;
+    private String[] addrs;
 
     /** Server addresses finder. */
     private transient ClientAddressFinder addrFinder;
@@ -122,7 +122,7 @@ public final class ClientConfiguration implements Serializable {
     private int reconnectThrottlingRetries = 3;
 
     /** Retry limit. */
-    private int retryLimit = 0;
+    private int retryLimit;
 
     /** Retry policy. */
     private ClientRetryPolicy retryPolicy = new ClientRetryAllPolicy();
@@ -136,6 +136,11 @@ public final class ClientConfiguration implements Serializable {
     /** Heartbeat interval, in milliseconds. */
     private long heartbeatInterval = 30_000L;
 
+    /**
+     * Whether automatic binary configuration should be enabled.
+     */
+    private boolean autoBinaryConfigurationEnabled = true;
+
     /**
      * @return Host addresses.
      */
@@ -748,4 +753,37 @@ public final class ClientConfiguration implements Serializable {
 
         return this;
     }
+
+    /**
+     * Gets a value indicating whether automatic binary configuration retrieval should be enabled.
+     * <p />
+     * When enabled, compact footer ({@link BinaryConfiguration#isCompactFooter()})
+     * and name mapper ({@link BinaryConfiguration#getNameMapper()}) settings will be retrieved from the server
+     * to match the cluster configuration.
+     * <p />
+     * Default is {@code true}.
+     *
+     * @return Whether automatic binary configuration is enabled.
+     */
+    public boolean isAutoBinaryConfigurationEnabled() {
+        return autoBinaryConfigurationEnabled;
+    }
+
+    /**
+     * Sets a value indicating whether automatic binary configuration retrieval should be enabled.
+     * <p />
+     * When enabled, compact footer ({@link BinaryConfiguration#isCompactFooter()})
+     * and name mapper ({@link BinaryConfiguration#getNameMapper()}) settings will be retrieved from the server
+     * to match the cluster configuration.
+     * <p />
+     * Default is {@code true}.
+     *
+     * @param autoBinaryConfigurationEnabled Whether automatic binary configuration is enabled.
+     * @return {@code this} for chaining.
+     */
+    public ClientConfiguration setAutoBinaryConfigurationEnabled(boolean autoBinaryConfigurationEnabled) {
+        this.autoBinaryConfigurationEnabled = autoBinaryConfigurationEnabled;
+
+        return this;
+    }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/BinaryNameMapperMode.java b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/BinaryNameMapperMode.java
new file mode 100644
index 00000000000..91303713b0b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/BinaryNameMapperMode.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.client.thin;
+
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Represents the binary name mapper mode.
+ */
+enum BinaryNameMapperMode {
+    /**
+     * Full name mapper.
+     */
+    BASIC_FULL,
+
+    /**
+     * Simple name mapper.
+     */
+    BASIC_SIMPLE,
+
+    /**
+     * Custom name mapper.
+     */
+    CUSTOM;
+
+    /** Enumerated values. */
+    private static final BinaryNameMapperMode[] VALS = values();
+
+    /**
+     * Gets enumerated value from its ordinal.
+     *
+     * @param ord Ordinal value.
+     * @return Enumerated value or {@code null} if ordinal out of range.
+     */
+    @Nullable
+    public static BinaryNameMapperMode fromOrdinal(int ord) {
+        return ord >= 0 && ord < VALS.length ? VALS[ord] : null;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientBinaryMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientBinaryMarshaller.java
index 65e4a7229fa..a8b5ba54abd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientBinaryMarshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientBinaryMarshaller.java
@@ -85,8 +85,7 @@ class ClientBinaryMarshaller {
      * Configure marshaller with custom Ignite Binary Object configuration.
      */
     public void setBinaryConfiguration(BinaryConfiguration binCfg) {
-        if (impl.context().configuration().getBinaryConfiguration() != binCfg)
-            impl = createImpl(binCfg);
+        impl = createImpl(binCfg);
     }
 
     /**
@@ -121,4 +120,3 @@ class ClientBinaryMarshaller {
         return new GridBinaryMarshaller(ctx);
     }
 }
-
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientChannelConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientChannelConfiguration.java
index 697d0eabbf9..a23d2226614 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientChannelConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientChannelConfiguration.java
@@ -102,6 +102,9 @@ final class ClientChannelConfiguration {
     /** Heartbeat interval, in milliseconds. */
     private final long heartbeatInterval;
 
+    /** Automatic binary configuration. */
+    private final boolean autoBinaryConfigurationEnabled;
+
     /**
      * Constructor.
      */
@@ -131,6 +134,7 @@ final class ClientChannelConfiguration {
         this.asyncContinuationExecutor = cfg.getAsyncContinuationExecutor();
         this.heartbeatEnabled = cfg.isHeartbeatEnabled();
         this.heartbeatInterval = cfg.getHeartbeatInterval();
+        this.autoBinaryConfigurationEnabled = cfg.isAutoBinaryConfigurationEnabled();
     }
 
     /**
@@ -301,4 +305,11 @@ final class ClientChannelConfiguration {
     public long getHeartbeatInterval() {
         return heartbeatInterval;
     }
+
+    /**
+     * @return Whether automatic binary configuration is enabled.
+     */
+    public boolean isAutoBinaryConfigurationEnabled() {
+        return autoBinaryConfigurationEnabled;
+    }
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientInternalBinaryConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientInternalBinaryConfiguration.java
new file mode 100644
index 00000000000..da8cb03f737
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientInternalBinaryConfiguration.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.client.thin;
+
+import org.apache.ignite.internal.binary.streams.BinaryInputStream;
+
+/**
+ * Cluster binary configuration.
+ */
+class ClientInternalBinaryConfiguration {
+    /** */
+    private final boolean compactFooter;
+
+    /** */
+    private final BinaryNameMapperMode binaryNameMapperMode;
+
+    /**
+     * Constructor.
+     *
+     * @param stream Stream.
+     */
+    public ClientInternalBinaryConfiguration(BinaryInputStream stream) {
+        compactFooter = stream.readBoolean();
+        binaryNameMapperMode = BinaryNameMapperMode.fromOrdinal(stream.readByte());
+    }
+
+    /**
+     * @return Compact footer.
+     */
+    public boolean compactFooter() {
+        return compactFooter;
+    }
+
+    /**
+     * @return Name mapper mode.
+     */
+    public BinaryNameMapperMode binaryNameMapperMode() {
+        return binaryNameMapperMode;
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientOperation.java b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientOperation.java
index 2d4af32a18b..20820666f27 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientOperation.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientOperation.java
@@ -144,17 +144,20 @@ public enum ClientOperation {
     /** Continuous query event. */
     QUERY_CONTINUOUS_EVENT(2007, ClientNotificationType.CONTINUOUS_QUERY_EVENT),
 
-    /** Get binary type. */
-    GET_BINARY_TYPE(3002),
+    /** Get binary type name. */
+    GET_BINARY_TYPE_NAME(3000),
 
     /** Register binary type name. */
     REGISTER_BINARY_TYPE_NAME(3001),
 
+    /** Get binary type. */
+    GET_BINARY_TYPE(3002),
+
     /** Put binary type. */
     PUT_BINARY_TYPE(3003),
 
-    /** Get binary type name. */
-    GET_BINARY_TYPE_NAME(3000),
+    /** Get binary configuration. */
+    GET_BINARY_CONFIGURATION(3004),
 
     /** Start new transaction. */
     TX_START(4000),
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ProtocolBitmaskFeature.java b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ProtocolBitmaskFeature.java
index ac5930da360..bf035f46ac6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ProtocolBitmaskFeature.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ProtocolBitmaskFeature.java
@@ -49,6 +49,9 @@ public enum ProtocolBitmaskFeature {
     /** Additional SqlFieldsQuery properties: partitions, updateBatchSize */
     QRY_PARTITIONS_BATCH_SIZE(7),
 
+    /** Binary configuration retrieval. */
+    BINARY_CONFIGURATION(8),
+
     /** Handle of {@link ClientServices#serviceDescriptors()}. */
     GET_SERVICE_DESCRIPTORS(9),
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ReliableChannel.java b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ReliableChannel.java
index 83173e3c125..f9529349d0e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ReliableChannel.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ReliableChannel.java
@@ -731,7 +731,7 @@ final class ReliableChannel implements AutoCloseable {
     }
 
     /** */
-    private <T> T applyOnDefaultChannel(Function<ClientChannel, T> function, ClientOperation op) {
+    <T> T applyOnDefaultChannel(Function<ClientChannel, T> function, ClientOperation op) {
         return applyOnDefaultChannel(function, op, getRetryLimit(), DO_NOTHING);
     }
 
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpIgniteClient.java b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpIgniteClient.java
index 3a3a2664fcc..8586ec01854 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpIgniteClient.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpIgniteClient.java
@@ -29,6 +29,7 @@ import java.util.function.BiFunction;
 import java.util.function.Consumer;
 import org.apache.ignite.IgniteBinary;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.binary.BinaryBasicNameMapper;
 import org.apache.ignite.binary.BinaryObjectException;
 import org.apache.ignite.binary.BinaryType;
 import org.apache.ignite.cache.query.FieldsQueryCursor;
@@ -45,6 +46,7 @@ import org.apache.ignite.client.ClientServices;
 import org.apache.ignite.client.ClientTransactions;
 import org.apache.ignite.client.IgniteClient;
 import org.apache.ignite.client.IgniteClientFuture;
+import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.ClientConfiguration;
 import org.apache.ignite.configuration.ClientTransactionConfiguration;
 import org.apache.ignite.internal.MarshallerPlatformIds;
@@ -59,6 +61,8 @@ import org.apache.ignite.internal.binary.BinaryWriterExImpl;
 import org.apache.ignite.internal.binary.streams.BinaryInputStream;
 import org.apache.ignite.internal.binary.streams.BinaryOutputStream;
 import org.apache.ignite.internal.client.thin.io.ClientConnectionMultiplexer;
+import org.apache.ignite.internal.processors.platform.client.ClientStatus;
+import org.apache.ignite.internal.processors.platform.client.IgniteClientException;
 import org.apache.ignite.internal.util.GridArgumentCheck;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgnitePredicate;
@@ -128,6 +132,8 @@ public class TcpIgniteClient implements IgniteClient {
         try {
             ch.channelsInit();
 
+            retrieveBinaryConfiguration(cfg);
+
             // Metadata, binary descriptors and user types caches must be cleared so that the
             // client will register all the user types within the cluster once again in case this information
             // was lost during the cluster failover.
@@ -412,6 +418,49 @@ public class TcpIgniteClient implements IgniteClient {
         }
     }
 
+    /** Load cluster binary configration. */
+    private void retrieveBinaryConfiguration(ClientConfiguration cfg) {
+        if (!cfg.isAutoBinaryConfigurationEnabled())
+            return;
+
+        ClientInternalBinaryConfiguration clusterCfg = ch.applyOnDefaultChannel(
+                c -> c.protocolCtx().isFeatureSupported(ProtocolBitmaskFeature.BINARY_CONFIGURATION)
+                ? c.service(ClientOperation.GET_BINARY_CONFIGURATION, null, r -> new ClientInternalBinaryConfiguration(r.in()))
+                : null,
+                ClientOperation.GET_BINARY_CONFIGURATION);
+
+        if (clusterCfg == null)
+            return;
+
+        BinaryConfiguration resCfg = cfg.getBinaryConfiguration();
+
+        if (resCfg == null)
+            resCfg = new BinaryConfiguration();
+
+        resCfg.setCompactFooter(clusterCfg.compactFooter());
+
+        switch (clusterCfg.binaryNameMapperMode()) {
+            case BASIC_FULL:
+                resCfg.setNameMapper(new BinaryBasicNameMapper().setSimpleName(false));
+                break;
+
+            case BASIC_SIMPLE:
+                resCfg.setNameMapper(new BinaryBasicNameMapper().setSimpleName(true));
+                break;
+
+            case CUSTOM:
+                if (resCfg.getNameMapper() == null || resCfg.getNameMapper() instanceof BinaryBasicNameMapper) {
+                    throw new IgniteClientException(ClientStatus.FAILED,
+                            "Custom binary name mapper is configured on the server, but not on the client. "
+                                    + "Update client BinaryConfigration to match the server.");
+                }
+
+                break;
+        }
+
+        marsh.setBinaryConfiguration(resCfg);
+    }
+
     /**
      * Thin client implementation of {@link BinaryMetadataHandler}.
      */
diff --git a/modules/core/src/test/java/org/apache/ignite/client/BinaryConfigurationTest.java b/modules/core/src/test/java/org/apache/ignite/client/BinaryConfigurationTest.java
new file mode 100644
index 00000000000..b09efe4601f
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/client/BinaryConfigurationTest.java
@@ -0,0 +1,175 @@
+/*
+ * 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.client;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.binary.BinaryBasicNameMapper;
+import org.apache.ignite.binary.BinaryNameMapper;
+import org.apache.ignite.configuration.BinaryConfiguration;
+import org.apache.ignite.configuration.ClientConfiguration;
+import org.apache.ignite.internal.binary.BinaryObjectImpl;
+import org.apache.ignite.internal.client.thin.AbstractThinClientTest;
+import org.apache.ignite.internal.processors.platform.client.IgniteClientException;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.junit.Test;
+
+import static org.apache.ignite.internal.binary.BinaryUtils.FLAG_COMPACT_FOOTER;
+
+/**
+ * Tests binary configuration behavior.
+ */
+public class BinaryConfigurationTest extends AbstractThinClientTest {
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+        super.afterTest();
+    }
+
+    /**
+     * Tests that client retrieves binary configuration from the server by default.
+     */
+    @Test
+    public void testAutoBinaryConfigurationEnabledRetrievesValuesFromServer() throws Exception {
+        Ignite server = startGrid(0);
+
+        try (IgniteClient client = startClient(0)) {
+            BinaryObjectImpl res = getClientBinaryObjectFromServer(server, client);
+
+            // Server-side defaults are compact footers and full name mapper.
+            assertTrue(res.isFlagSet(FLAG_COMPACT_FOOTER));
+            assertEquals("org.apache.ignite.client.Person", res.type().typeName());
+        }
+    }
+
+    /**
+     * Tests that client retrieves binary configuration from the server and overrides client configuration settings.
+     */
+    @Test
+    public void testAutoBinaryConfigurationEnabledOverridesExplicitClientSettings() throws Exception {
+        Ignite server = startGrid(0);
+
+        BinaryConfiguration binaryCfg = new BinaryConfiguration()
+                .setCompactFooter(false)
+                .setNameMapper(new BinaryBasicNameMapper().setSimpleName(true));
+
+        ClientConfiguration clientCfg = getClientConfiguration(server)
+                .setBinaryConfiguration(binaryCfg);
+
+        try (IgniteClient client = Ignition.startClient(clientCfg)) {
+            BinaryObjectImpl res = getClientBinaryObjectFromServer(server, client);
+
+            // Server-side defaults are compact footers and full name mapper.
+            assertTrue(res.isFlagSet(FLAG_COMPACT_FOOTER));
+            assertEquals("org.apache.ignite.client.Person", res.type().typeName());
+        }
+    }
+
+    /**
+     * Tests that client does not retrieve binary configuration from the server when this behavior is disabled.
+     */
+    @Test
+    public void testAutoBinaryConfigurationDisabledKeepsClientSettingsAsIs() throws Exception {
+        Ignite server = startGrid(0);
+
+        BinaryConfiguration binaryCfg = new BinaryConfiguration()
+                .setCompactFooter(false)
+                .setNameMapper(new BinaryBasicNameMapper().setSimpleName(true));
+
+        ClientConfiguration clientCfg = getClientConfiguration(server)
+                .setAutoBinaryConfigurationEnabled(false)
+                .setBinaryConfiguration(binaryCfg);
+
+        try (IgniteClient client = Ignition.startClient(clientCfg)) {
+            BinaryObjectImpl res = getClientBinaryObjectFromServer(server, client);
+
+            assertFalse(res.isFlagSet(FLAG_COMPACT_FOOTER));
+            assertEquals("Person", res.type().typeName());
+        }
+    }
+
+    /**
+     * Tests that client throws an exception on start when server has a custom mapper configured, but client has not.
+     */
+    @Test
+    public void testCustomMapperOnServerDefaultMapperOnClientThrows() throws Exception {
+        BinaryConfiguration serverBinaryCfg = new BinaryConfiguration()
+                .setNameMapper(new CustomBinaryNameMapper());
+
+        Ignite server = startGrid("0", cfg -> cfg.setBinaryConfiguration(serverBinaryCfg));
+
+        BinaryConfiguration binaryCfg = new BinaryConfiguration()
+                .setNameMapper(new BinaryBasicNameMapper());
+
+        ClientConfiguration clientCfg = getClientConfiguration(server)
+                .setBinaryConfiguration(binaryCfg);
+
+        GridTestUtils.assertThrowsAnyCause(null, () -> Ignition.startClient(clientCfg), IgniteClientException.class,
+                "Custom binary name mapper is configured on the server, but not on the client."
+                        + " Update client BinaryConfigration to match the server.");
+    }
+
+    /**
+     * Tests that client works as expected when custom mapper is configured on both sides.
+     */
+    @Test
+    public void testCustomMapperOnServerCustomMapperOnClientDoesNotThrow() throws Exception {
+        BinaryConfiguration binaryCfg = new BinaryConfiguration()
+                .setNameMapper(new CustomBinaryNameMapper());
+
+        Ignite server = startGrid("0", cfg -> cfg.setBinaryConfiguration(binaryCfg));
+
+        ClientConfiguration clientCfg = getClientConfiguration(server)
+                .setBinaryConfiguration(binaryCfg);
+
+        try (IgniteClient client = Ignition.startClient(clientCfg)) {
+            BinaryObjectImpl res = getClientBinaryObjectFromServer(server, client);
+
+            assertTrue(res.isFlagSet(FLAG_COMPACT_FOOTER));
+            assertEquals("org.apache.ignite.client.Person_", res.type().typeName());
+        }
+    }
+
+    /**
+     * Inserts an object from the client and retrieves it as a binary object from the server.
+     *
+     * @param server Server.
+     * @param client Client.
+     * @return Binary object.
+     */
+    private BinaryObjectImpl getClientBinaryObjectFromServer(Ignite server, IgniteClient client) {
+        client.getOrCreateCache("c").put(1, new Person(1, "1"));
+
+        return server.cache("c").<Integer, BinaryObjectImpl>withKeepBinary().get(1);
+    }
+
+    /**
+     * Custom mapper.
+     */
+    private static class CustomBinaryNameMapper implements BinaryNameMapper {
+        /** {@inheritDoc} */
+        @Override public String typeName(String clsName) {
+            return clsName + "_";
+        }
+
+        /** {@inheritDoc} */
+        @Override public String fieldName(String fieldName) {
+            return fieldName + "!";
+        }
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/client/ClientConfigurationTest.java b/modules/core/src/test/java/org/apache/ignite/client/ClientConfigurationTest.java
index 623b7fc9a21..8f882653b09 100644
--- a/modules/core/src/test/java/org/apache/ignite/client/ClientConfigurationTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/client/ClientConfigurationTest.java
@@ -72,6 +72,7 @@ public class ClientConfigurationTest {
             .setSslTrustCertificateKeyStorePassword("123456")
             .setSslKeyAlgorithm(DFLT_KEY_ALGORITHM)
             .setHeartbeatInterval(3000)
+            .setAutoBinaryConfigurationEnabled(false)
             .setHeartbeatEnabled(true);
 
         ByteArrayOutputStream outBytes = new ByteArrayOutputStream();
diff --git a/modules/core/src/test/java/org/apache/ignite/client/ReliabilityTest.java b/modules/core/src/test/java/org/apache/ignite/client/ReliabilityTest.java
index f6457f69623..aaeabdf0a03 100644
--- a/modules/core/src/test/java/org/apache/ignite/client/ReliabilityTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/client/ReliabilityTest.java
@@ -316,7 +316,7 @@ public class ReliabilityTest extends AbstractThinClientTest {
 
         String nullOpsNames = nullOps.stream().map(Enum::name).collect(Collectors.joining(", "));
 
-        long expectedNullCount = 14;
+        long expectedNullCount = 15;
 
         String msg = nullOps.size()
                 + " operation codes do not have public equivalent. When adding new codes, update ClientOperationType too. Missing ops: "
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/client/thin/AbstractThinClientTest.java b/modules/core/src/test/java/org/apache/ignite/internal/client/thin/AbstractThinClientTest.java
index 0e33127d961..ecf394d1a33 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/client/thin/AbstractThinClientTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/client/thin/AbstractThinClientTest.java
@@ -40,6 +40,34 @@ public abstract class AbstractThinClientTest extends GridCommonAbstractTest {
         return new ClientConfiguration().setPartitionAwarenessEnabled(false);
     }
 
+    /**
+     * Gets default client configuration with addresses set to the specified nodes.
+     *
+     * @param nodes Server nodes.
+     */
+    protected ClientConfiguration getClientConfiguration(ClusterNode... nodes) {
+        String[] addrs = new String[nodes.length];
+
+        for (int i = 0; i < nodes.length; i++) {
+            ClusterNode node = nodes[i];
+
+            addrs[i] = clientHost(node) + ":" + clientPort(node);
+        }
+
+        return getClientConfiguration().setAddresses(addrs);
+    }
+
+    /**
+     * Gets default client configuration with addresses set to the specified nodes.
+     *
+     * @param ignites Server nodes.
+     */
+    protected ClientConfiguration getClientConfiguration(Ignite... ignites) {
+        ClusterNode[] nodes = Arrays.stream(ignites).map(ignite -> ignite.cluster().localNode()).toArray(ClusterNode[]::new);
+
+        return getClientConfiguration(nodes);
+    }
+
     /**
      * Return thin client port for given node.
      *
@@ -65,15 +93,9 @@ public abstract class AbstractThinClientTest extends GridCommonAbstractTest {
      * @return Thin client.
      */
     protected IgniteClient startClient(ClusterNode... nodes) {
-        String[] addrs = new String[nodes.length];
-
-        for (int i = 0; i < nodes.length; i++) {
-            ClusterNode node = nodes[i];
-
-            addrs[i] = clientHost(node) + ":" + clientPort(node);
-        }
+        ClientConfiguration cfg = getClientConfiguration(nodes);
 
-        return Ignition.startClient(getClientConfiguration().setAddresses(addrs));
+        return Ignition.startClient(cfg);
     }
 
     /**
diff --git a/modules/indexing/src/test/java/org/apache/ignite/client/ClientTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/client/ClientTestSuite.java
index d21bff31efc..7234a85361e 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/client/ClientTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/client/ClientTestSuite.java
@@ -70,7 +70,8 @@ import org.junit.runners.Suite;
     CacheAsyncTest.class,
     TimeoutTest.class,
     OptimizedMarshallerClassesCachedTest.class,
-    AtomicLongTest.class
+    AtomicLongTest.class,
+    BinaryConfigurationTest.class
 })
 public class ClientTestSuite {
     // No-op.