[
https://issues.apache.org/jira/browse/IGNITE-16186?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Andrey Belyaev updated IGNITE-16186:
------------------------------------
Description:
When creating ignite cache, you can set parameter _CacheKeyConfiguration_ in
_CacheConfiguration_ and specify there binary type and affinity field which
will be used for placing data through partitions.
But there is a problem with such configuring. The information about affinity is
out of binary context, i.e. affinity field is missing for specified key type.
This situation can be partly corrected by setting same config in
{_}IgniteConfiguration#setCacheKeyConfiguration(){_}. Then binary type to
affinity field binding information explictly places in binary context map at
grid startup. Unfortunately, this method doesn't help for runtime created
caches.
This problem with binary context can produce issue if you first put in cache
binary object (via _cache#put_ api) and then do sql insert.
Code example:
{code:java}
try (Ignite ignite = Ignition.start(new IgniteConfiguration())) {
LinkedHashMap<String, String> fields = new LinkedHashMap<>();
fields.put("A", "java.lang.String");
fields.put("B", "java.lang.String");
fields.put("C", "java.lang.String");
Set<String> keyFields = new LinkedHashSet<>();
keyFields.add("A");
keyFields.add("B");
CacheConfiguration<Object, Object> cacheConfiguration = new
CacheConfiguration<>()
.setName("JOINDATACACHE")
.setKeyConfiguration(new CacheKeyConfiguration()
.setTypeName("BinaryPojoKey")
.setAffinityKeyFieldName("B"))
.setQueryEntities(Collections.singleton(new QueryEntity()
.setTableName("JOINDATACACHE")
.setKeyType("BinaryPojoKey")
.setValueType("BinaryPojoValue")
.setFields(fields)
.setKeyFields(keyFields)));
IgniteCache<Object, Object> cache =
ignite.getOrCreateCache(cacheConfiguration);
// putBinary
BinaryObjectBuilder keyBuilder = ignite.binary().builder("BinaryPojoKey");
keyBuilder.setField("B", "B_VAL", String.class);
keyBuilder.setField("A", "A_VAL", String.class);
BinaryObjectBuilder valueBuilder =
ignite.binary().builder("BinaryPojoValue");
valueBuilder.setField("C", "C_VAL", String.class);
cache.withKeepBinary().put(keyBuilder.build(), valueBuilder.build());
// putSql produce error
cache.query(new SqlFieldsQuery("INSERT INTO JOINDATACACHE (A, B, C) VALUES
('A_VAL_1', 'B_VAL_1', 'C_VAL_1')"));
} {code}
Code in 37 with sql insertion line produce exception:
{code:java}
Exception in thread "main" javax.cache.CacheException: Binary type has
different affinity key fields [typeName=BinaryPojoKey, affKeyFieldName1=null,
affKeyFieldName2=B]
at
org.apache.ignite.internal.processors.cache.IgniteCacheProxyImpl.query(IgniteCacheProxyImpl.java:854)
at
org.apache.ignite.internal.processors.cache.IgniteCacheProxyImpl.query(IgniteCacheProxyImpl.java:787)
at
org.apache.ignite.internal.processors.cache.GatewayProtectedCacheProxy.query(GatewayProtectedCacheProxy.java:430)
at com.gridgain.investigate.App.main(App.java:65)
Caused by: class org.apache.ignite.binary.BinaryObjectException: Binary type
has different affinity key fields [typeName=BinaryPojoKey,
affKeyFieldName1=null, affKeyFieldName2=B]
at
org.apache.ignite.internal.binary.BinaryUtils.mergeMetadata(BinaryUtils.java:1008)
at
org.apache.ignite.internal.processors.cache.binary.BinaryMetadataTransport.requestMetadataUpdate(BinaryMetadataTransport.java:211)
at
org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl.addMeta(CacheObjectBinaryProcessorImpl.java:639)
at
org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl$1.addMeta(CacheObjectBinaryProcessorImpl.java:292)
at
org.apache.ignite.internal.binary.BinaryContext.updateMetadata(BinaryContext.java:1346)
at
org.apache.ignite.internal.binary.builder.BinaryObjectBuilderImpl.serializeTo(BinaryObjectBuilderImpl.java:369)
at
org.apache.ignite.internal.binary.builder.BinaryObjectBuilderImpl.build(BinaryObjectBuilderImpl.java:192)
at
org.apache.ignite.internal.processors.query.h2.dml.UpdatePlan.processRow(UpdatePlan.java:282)
at
org.apache.ignite.internal.processors.query.h2.dml.DmlUtils.dmlDoInsert(DmlUtils.java:202)
at
org.apache.ignite.internal.processors.query.h2.dml.DmlUtils.processSelectResult(DmlUtils.java:175)
at
org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.executeUpdateNonTransactional(IgniteH2Indexing.java:2855)
at
org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.executeUpdate(IgniteH2Indexing.java:2705)
at
org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.executeUpdateDistributed(IgniteH2Indexing.java:2635)
at
org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.executeDml(IgniteH2Indexing.java:1240)
at
org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.querySqlFields(IgniteH2Indexing.java:1162)
at
org.apache.ignite.internal.processors.query.GridQueryProcessor$2.applyx(GridQueryProcessor.java:2883)
at
org.apache.ignite.internal.processors.query.GridQueryProcessor$2.applyx(GridQueryProcessor.java:2879)
at
org.apache.ignite.internal.util.lang.IgniteOutClosureX.apply(IgniteOutClosureX.java:36)
at
org.apache.ignite.internal.processors.query.GridQueryProcessor.executeQuery(GridQueryProcessor.java:3478)
at
org.apache.ignite.internal.processors.query.GridQueryProcessor.lambda$querySqlFields$3(GridQueryProcessor.java:2899)
at
org.apache.ignite.internal.processors.query.GridQueryProcessor.executeQuerySafe(GridQueryProcessor.java:2934)
at
org.apache.ignite.internal.processors.query.GridQueryProcessor.querySqlFields(GridQueryProcessor.java:2873)
at
org.apache.ignite.internal.processors.query.GridQueryProcessor.querySqlFields(GridQueryProcessor.java:2800)
at
org.apache.ignite.internal.processors.cache.IgniteCacheProxyImpl.query(IgniteCacheProxyImpl.java:839)
... 3 more {code}
was:
When creating ignite cache, you can set parameter _CacheKeyConfiguration_ in
_CacheConfiguration_ and specify there binary type and affinity field which
will be used for placing data through partitions.
But there is a problem with such configuring. The information about affinity is
out of binary context, i.e. affinity field is missing for specified key type.
This situation can be partly corrected by setting same config in
{_}IgniteConfiguration#setCacheKeyConfiguration(){_}. Then binary type to
affinity field binding information explictly places in binary context map at
grid startup. Unfortunately, this method doesn't help for runtime created
caches.
This problem with binary context can produce issue if you first put in cache
binary object (via _cache#put_ api) and then do sql insert.
Code example:
{code:java}
try (Ignite ignite = Ignition.start(new IgniteConfiguration())) {
LinkedHashMap<String, String> fields = new LinkedHashMap<>();
fields.put("A", "java.lang.String");
fields.put("B", "java.lang.String");
fields.put("C", "java.lang.String");
Set<String> keyFields = new LinkedHashSet<>();
keyFields.add("A");
keyFields.add("B");
CacheConfiguration<Object, Object> cacheConfiguration = new
CacheConfiguration<>()
.setName("JOINDATACACHE")
.setKeyConfiguration(new CacheKeyConfiguration()
.setTypeName("BinaryPojoKey")
.setAffinityKeyFieldName("B"))
.setQueryEntities(Collections.singleton(new QueryEntity()
.setTableName("JOINDATACACHE")
.setKeyType("BinaryPojoKey")
.setValueType("BinaryPojoValue")
.setFields(fields)
.setKeyFields(keyFields)));
IgniteCache<Object, Object> cache =
ignite.getOrCreateCache(cacheConfiguration);
// putBinary
BinaryObjectBuilder keyBuilder = ignite.binary().builder("BinaryPojoKey");
keyBuilder.setField("B", "B_VAL", String.class);
keyBuilder.setField("A", "A_VAL", String.class);
BinaryObjectBuilder valueBuilder =
ignite.binary().builder("BinaryPojoValue");
valueBuilder.setField("C", "C_VAL", String.class);
cache.withKeepBinary().put(keyBuilder.build(), valueBuilder.build());
// putSql produce error
cache.query(new SqlFieldsQuery("INSERT INTO JOINDATACACHE (A, B, C) VALUES
('A_VAL_1', 'B_VAL_1', 'C_VAL_1')"));
} {code}
Code in 37 with sql insertion line produce exception:
{code:java}
Exception in thread "main" javax.cache.CacheException: Binary type has
different affinity key fields [typeName=BinaryPojoKey, affKeyFieldName1=null,
affKeyFieldName2=B]
at
org.apache.ignite.internal.processors.cache.IgniteCacheProxyImpl.query(IgniteCacheProxyImpl.java:854)
at
org.apache.ignite.internal.processors.cache.IgniteCacheProxyImpl.query(IgniteCacheProxyImpl.java:787)
at
org.apache.ignite.internal.processors.cache.GatewayProtectedCacheProxy.query(GatewayProtectedCacheProxy.java:430)
at com.gridgain.investigate.App.main(App.java:65)
Caused by: class org.apache.ignite.binary.BinaryObjectException: Binary type
has different affinity key fields [typeName=BinaryPojoKey,
affKeyFieldName1=null, affKeyFieldName2=B]
at
org.apache.ignite.internal.binary.BinaryUtils.mergeMetadata(BinaryUtils.java:1008)
at
org.apache.ignite.internal.processors.cache.binary.BinaryMetadataTransport.requestMetadataUpdate(BinaryMetadataTransport.java:211)
at
org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl.addMeta(CacheObjectBinaryProcessorImpl.java:639)
at
org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl$1.addMeta(CacheObjectBinaryProcessorImpl.java:292)
at
org.apache.ignite.internal.binary.BinaryContext.updateMetadata(BinaryContext.java:1346)
at
org.apache.ignite.internal.binary.builder.BinaryObjectBuilderImpl.serializeTo(BinaryObjectBuilderImpl.java:369)
at
org.apache.ignite.internal.binary.builder.BinaryObjectBuilderImpl.build(BinaryObjectBuilderImpl.java:192)
at
org.apache.ignite.internal.processors.query.h2.dml.UpdatePlan.processRow(UpdatePlan.java:282)
at
org.apache.ignite.internal.processors.query.h2.dml.DmlUtils.dmlDoInsert(DmlUtils.java:202)
at
org.apache.ignite.internal.processors.query.h2.dml.DmlUtils.processSelectResult(DmlUtils.java:175)
at
org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.executeUpdateNonTransactional(IgniteH2Indexing.java:2855)
at
org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.executeUpdate(IgniteH2Indexing.java:2705)
at
org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.executeUpdateDistributed(IgniteH2Indexing.java:2635)
at
org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.executeDml(IgniteH2Indexing.java:1240)
at
org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.querySqlFields(IgniteH2Indexing.java:1162)
at
org.apache.ignite.internal.processors.query.GridQueryProcessor$2.applyx(GridQueryProcessor.java:2883)
at
org.apache.ignite.internal.processors.query.GridQueryProcessor$2.applyx(GridQueryProcessor.java:2879)
at
org.apache.ignite.internal.util.lang.IgniteOutClosureX.apply(IgniteOutClosureX.java:36)
at
org.apache.ignite.internal.processors.query.GridQueryProcessor.executeQuery(GridQueryProcessor.java:3478)
at
org.apache.ignite.internal.processors.query.GridQueryProcessor.lambda$querySqlFields$3(GridQueryProcessor.java:2899)
at
org.apache.ignite.internal.processors.query.GridQueryProcessor.executeQuerySafe(GridQueryProcessor.java:2934)
at
org.apache.ignite.internal.processors.query.GridQueryProcessor.querySqlFields(GridQueryProcessor.java:2873)
at
org.apache.ignite.internal.processors.query.GridQueryProcessor.querySqlFields(GridQueryProcessor.java:2800)
at
org.apache.ignite.internal.processors.cache.IgniteCacheProxyImpl.query(IgniteCacheProxyImpl.java:839)
... 3 more {code}
> Possibility of declaring affinity key fields for binary object cache key
> ------------------------------------------------------------------------
>
> Key: IGNITE-16186
> URL: https://issues.apache.org/jira/browse/IGNITE-16186
> Project: Ignite
> Issue Type: Bug
> Reporter: Andrey Belyaev
> Assignee: Andrey Belyaev
> Priority: Major
>
> When creating ignite cache, you can set parameter _CacheKeyConfiguration_ in
> _CacheConfiguration_ and specify there binary type and affinity field which
> will be used for placing data through partitions.
> But there is a problem with such configuring. The information about affinity
> is out of binary context, i.e. affinity field is missing for specified key
> type.
> This situation can be partly corrected by setting same config in
> {_}IgniteConfiguration#setCacheKeyConfiguration(){_}. Then binary type to
> affinity field binding information explictly places in binary context map at
> grid startup. Unfortunately, this method doesn't help for runtime created
> caches.
> This problem with binary context can produce issue if you first put in cache
> binary object (via _cache#put_ api) and then do sql insert.
> Code example:
> {code:java}
> try (Ignite ignite = Ignition.start(new IgniteConfiguration())) {
> LinkedHashMap<String, String> fields = new LinkedHashMap<>();
> fields.put("A", "java.lang.String");
> fields.put("B", "java.lang.String");
> fields.put("C", "java.lang.String");
> Set<String> keyFields = new LinkedHashSet<>();
> keyFields.add("A");
> keyFields.add("B");
> CacheConfiguration<Object, Object> cacheConfiguration = new
> CacheConfiguration<>()
> .setName("JOINDATACACHE")
> .setKeyConfiguration(new CacheKeyConfiguration()
> .setTypeName("BinaryPojoKey")
> .setAffinityKeyFieldName("B"))
> .setQueryEntities(Collections.singleton(new QueryEntity()
> .setTableName("JOINDATACACHE")
> .setKeyType("BinaryPojoKey")
> .setValueType("BinaryPojoValue")
> .setFields(fields)
> .setKeyFields(keyFields)));
> IgniteCache<Object, Object> cache =
> ignite.getOrCreateCache(cacheConfiguration);
> // putBinary
> BinaryObjectBuilder keyBuilder = ignite.binary().builder("BinaryPojoKey");
> keyBuilder.setField("B", "B_VAL", String.class);
> keyBuilder.setField("A", "A_VAL", String.class);
> BinaryObjectBuilder valueBuilder =
> ignite.binary().builder("BinaryPojoValue");
> valueBuilder.setField("C", "C_VAL", String.class);
> cache.withKeepBinary().put(keyBuilder.build(), valueBuilder.build());
> // putSql produce error
> cache.query(new SqlFieldsQuery("INSERT INTO JOINDATACACHE (A, B, C)
> VALUES ('A_VAL_1', 'B_VAL_1', 'C_VAL_1')"));
> } {code}
> Code in 37 with sql insertion line produce exception:
> {code:java}
> Exception in thread "main" javax.cache.CacheException: Binary type has
> different affinity key fields [typeName=BinaryPojoKey, affKeyFieldName1=null,
> affKeyFieldName2=B]
> at
> org.apache.ignite.internal.processors.cache.IgniteCacheProxyImpl.query(IgniteCacheProxyImpl.java:854)
> at
> org.apache.ignite.internal.processors.cache.IgniteCacheProxyImpl.query(IgniteCacheProxyImpl.java:787)
> at
> org.apache.ignite.internal.processors.cache.GatewayProtectedCacheProxy.query(GatewayProtectedCacheProxy.java:430)
> at com.gridgain.investigate.App.main(App.java:65)
> Caused by: class org.apache.ignite.binary.BinaryObjectException: Binary type
> has different affinity key fields [typeName=BinaryPojoKey,
> affKeyFieldName1=null, affKeyFieldName2=B]
> at
> org.apache.ignite.internal.binary.BinaryUtils.mergeMetadata(BinaryUtils.java:1008)
> at
> org.apache.ignite.internal.processors.cache.binary.BinaryMetadataTransport.requestMetadataUpdate(BinaryMetadataTransport.java:211)
> at
> org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl.addMeta(CacheObjectBinaryProcessorImpl.java:639)
> at
> org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl$1.addMeta(CacheObjectBinaryProcessorImpl.java:292)
> at
> org.apache.ignite.internal.binary.BinaryContext.updateMetadata(BinaryContext.java:1346)
> at
> org.apache.ignite.internal.binary.builder.BinaryObjectBuilderImpl.serializeTo(BinaryObjectBuilderImpl.java:369)
> at
> org.apache.ignite.internal.binary.builder.BinaryObjectBuilderImpl.build(BinaryObjectBuilderImpl.java:192)
> at
> org.apache.ignite.internal.processors.query.h2.dml.UpdatePlan.processRow(UpdatePlan.java:282)
> at
> org.apache.ignite.internal.processors.query.h2.dml.DmlUtils.dmlDoInsert(DmlUtils.java:202)
> at
> org.apache.ignite.internal.processors.query.h2.dml.DmlUtils.processSelectResult(DmlUtils.java:175)
> at
> org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.executeUpdateNonTransactional(IgniteH2Indexing.java:2855)
> at
> org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.executeUpdate(IgniteH2Indexing.java:2705)
> at
> org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.executeUpdateDistributed(IgniteH2Indexing.java:2635)
> at
> org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.executeDml(IgniteH2Indexing.java:1240)
> at
> org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.querySqlFields(IgniteH2Indexing.java:1162)
> at
> org.apache.ignite.internal.processors.query.GridQueryProcessor$2.applyx(GridQueryProcessor.java:2883)
> at
> org.apache.ignite.internal.processors.query.GridQueryProcessor$2.applyx(GridQueryProcessor.java:2879)
> at
> org.apache.ignite.internal.util.lang.IgniteOutClosureX.apply(IgniteOutClosureX.java:36)
> at
> org.apache.ignite.internal.processors.query.GridQueryProcessor.executeQuery(GridQueryProcessor.java:3478)
> at
> org.apache.ignite.internal.processors.query.GridQueryProcessor.lambda$querySqlFields$3(GridQueryProcessor.java:2899)
> at
> org.apache.ignite.internal.processors.query.GridQueryProcessor.executeQuerySafe(GridQueryProcessor.java:2934)
> at
> org.apache.ignite.internal.processors.query.GridQueryProcessor.querySqlFields(GridQueryProcessor.java:2873)
> at
> org.apache.ignite.internal.processors.query.GridQueryProcessor.querySqlFields(GridQueryProcessor.java:2800)
> at
> org.apache.ignite.internal.processors.cache.IgniteCacheProxyImpl.query(IgniteCacheProxyImpl.java:839)
> ... 3 more {code}
>
--
This message was sent by Atlassian Jira
(v8.20.1#820001)