Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 98DB9200B49 for ; Tue, 19 Jul 2016 12:52:22 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 977E5160A8B; Tue, 19 Jul 2016 10:52:22 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id E078F160A8C for ; Tue, 19 Jul 2016 12:52:21 +0200 (CEST) Received: (qmail 3149 invoked by uid 500); 19 Jul 2016 10:52:20 -0000 Mailing-List: contact issues-help@ignite.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ignite.apache.org Delivered-To: mailing list issues@ignite.apache.org Received: (qmail 3072 invoked by uid 99); 19 Jul 2016 10:52:20 -0000 Received: from arcas.apache.org (HELO arcas) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 19 Jul 2016 10:52:20 +0000 Received: from arcas.apache.org (localhost [127.0.0.1]) by arcas (Postfix) with ESMTP id 8F6442C0D5F for ; Tue, 19 Jul 2016 10:52:20 +0000 (UTC) Date: Tue, 19 Jul 2016 10:52:20 +0000 (UTC) From: "Alexei Scherbakov (JIRA)" To: issues@ignite.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Created] (IGNITE-3505) BinaryObject keys can't be reused because of partition caching. MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Tue, 19 Jul 2016 10:52:22 -0000 Alexei Scherbakov created IGNITE-3505: ----------------------------------------- Summary: BinaryObject keys can't be reused because of partition caching. Key: IGNITE-3505 URL: https://issues.apache.org/jira/browse/IGNITE-3505 Project: Ignite Issue Type: Bug Affects Versions: 1.6 Reporter: Alexei Scherbakov Assignee: Denis Magda Fix For: 1.7 BinaryObject can't be reused as key between caches because it's actual implementation BinaryObjectImpl implements KeyCacheObject and due to the fact caches partition, which is not recalculated later. See org.apache.ignite.internal.processors.cache.GridCacheAffinityManager.partition: {code} if (key instanceof KeyCacheObject && ((KeyCacheObject)key).partition() != -1) return ((KeyCacheObject)key).partition(); {code} The issue can be reproduced with the following code: {code} public static void main(String[] args) throws IgniteException { IgniteConfiguration cfg = new IgniteConfiguration(); cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(new TcpDiscoveryVmIpFinder(true))); Ignite ignite = Ignition.start(cfg); CacheConfiguration cfg1 = new CacheConfiguration<>("Cache 1"); cfg1.setCacheMode(CacheMode.PARTITIONED); cfg1.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL); IgniteCache cache1 = ignite.getOrCreateCache(cfg1).withKeepBinary(); CacheConfiguration cfg2 = new CacheConfiguration<>("Cache 2"); cfg2.setCacheMode(CacheMode.REPLICATED); cfg2.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); IgniteCache cache2 = ignite.getOrCreateCache(cfg2); BinaryObjectBuilder keyBuilder = ignite.binary().builder("keyType") .setField("F1", "V1").hashCode("V1".hashCode()); BinaryObjectBuilder valBuilder = ignite.binary().builder("valueType") .setField("F2", "V2") .setField("F3", "V3"); BinaryObject key = keyBuilder.build(); BinaryObject val = valBuilder.build(); cache1.put(key, val); cache2.put(key, val); // error System.out.println(cache1.get(key)); // error System.out.println(cache2.get(key)); } {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)