Hi Roger,

Please see inline

> On Sep 5, 2017, at 9:44 AM, Roger Fischer (CW) <rfis...@brocade.com> wrote:
> 
> Hi Denis,
>  
> I don’t quite understand your comment “If you use Strings as the keys you 
> won’t get affinity collocation set up properly”.
>  
> I have an object with a plain String key, named switchId, and another object 
> with a composite key, of which one field is switchId (of type String). I am 
> using switchId as the affinity key, and it seems to work fine without 
> distributed queries. Was this a coincident and I should re-test?
>  

Your understanding is correct. I thought you didn’t use the composite key.

> My understanding is that caches with no explicit affinity key use the key to 
> distribute objects. For the plain-key cache this would be switchId.
>  
> So, if I have two caches, both with switchId as the key, I should get 
> automatic collocation (no setup required). Correct?
>  

Yes if the AffinityFunction is the same across the caches.

> And for the third cache, with the switchId field in the composite key as the 
> affinity key, I should also get collocation (with the other two caches). 
> Correct?

Yes

>  
> What am I missing?

Make sure cacheKeyConfiguration is set for every cache cache with the affinity 
key you have. Alternatively you can try @AffinityKeyMapped annotation by 
labeling required field. If nothing works please share a reproducer via GitHub 
with us.

>  
> BTW, I think there was a typo in the response (first sentence), and you meant 
> to say “set up properly and _non_-distributed joins will return an incomplete 
> result.”.
>  

I meant _collocated_ joins. In Ignite all the joins are considered to be 
distributed in a sense that queries are executed in the distributed fashion.

—
Denis 

> Roger
>  
> PS: My configuration (all XML):
>  
>         <property name="cacheKeyConfiguration">
>             <list>
>                 <!-- ports cache -->
>                 <bean class="org.apache.ignite.cache.CacheKeyConfiguration">
>                     <property name="typeName" 
> value="com.abc.poc.inppoc.model.PortKey"/>
>                     <property name="affinityKeyFieldName" value="switchId"/>
>                 </bean>
>                 <!-- switches cache uses default affinity key (id) -->
>             </list>
>         </property>
>  
>  
>  
>                 <!-- ports cache -->
>                 <bean 
> class="org.apache.ignite.configuration.CacheConfiguration">
>                     <property name="name" value="PortCache"/>
>                     <property name="cacheMode" value="PARTITIONED"/>
>                     <property name="backups" value="1"/>
>  
>                     <!-- use affinity (on Switch ID) -->
>                     <property name="affinity">
>                         <bean 
> class="org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction">
>                             <property name="partitions" value="1024"/>
>                         </bean>
>                     </property>
>  
>                     <!-- Port fields/columns -->
>                     <property name="queryEntities">
>                         <list>
>                             <bean class="org.apache.ignite.cache.QueryEntity">
>                                 <property name="keyType" 
> value="com.abc.poc.inppoc.model.PortKey"/>
>                                 <property name="valueType" 
> value="com.abc.poc.inppoc.model.Port"/>
>                                 <property name="fields">
>                                     …
>  
>  
>                 <!-- switches cache -->
>                 <bean 
> class="org.apache.ignite.configuration.CacheConfiguration">
>                     <property name="name" value="SwitchCache"/>
>                     <property name="cacheMode" value="PARTITIONED"/>
>                     <property name="backups" value="1"/>
>  
>                     <!-- use affinity (on Switch ID, ie primary key) -->
>                     <property name="affinity">
>                         <bean 
> class="org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction">
>                             <property name="partitions" value="1024"/>
>                         </bean>
>                     </property>
>  
>                     <!-- Switch fields/columns -->
>                     <property name="queryEntities">
>                         <list>
>                             <bean class="org.apache.ignite.cache.QueryEntity">
>                                 <property name="keyType" 
> value="java.util.UUID"/>
>                                 <property name="valueType" 
> value="com.brocade.poc.inppoc.model.Switch"/>
>                                 <property name="fields">
>                                     …
>  
> PortKey.java:
>  
> public class PortKey implements Serializable {
>  
>     private UUID    id;        // port-id; PK
>     private UUID    switchId;  // affinity key; not really part of PK
>  
>     …
>  
>  
>  
> From: Denis Magda [mailto:dma...@apache.org <mailto:dma...@apache.org>] 
> Sent: Friday, September 01, 2017 3:07 PM
> To: user@ignite.apache.org <mailto:user@ignite.apache.org>
> Subject: Re: SQLQuery with simple Join return no results
>  
> If you use Strings as the keys you won’t get affinity collocation set up 
> properly and distributed joins will return an incomplete result. One of the 
> keys have to comprise a “parent” class key that will be an affinity key. Look 
> at the example here:
> https://apacheignite.readme.io/docs/affinity-collocation#section-collocate-data-with-data
>  
> <https://urldefense.proofpoint.com/v2/url?u=https-3A__apacheignite.readme.io_docs_affinity-2Dcollocation-23section-2Dcollocate-2Ddata-2Dwith-2Ddata&d=DwMFaQ&c=IL_XqQWOjubgfqINi2jTzg&r=1esZO0r0bYS90lcsaLA6N4AFxuNo6lzauhETGwdJQoQ&m=KTlx1Z0HYXpMSZKonCG-plYFaijMpCt0OUS3tJgKbWQ&s=-YW5SFh-ko4-_EBwKFMKH5BOJU7pGcQMqaCRZrwZz24&e=>
>  
> As for the NON collocated joins suggested by Roger (qry.setDistributedJoins( 
> true)), I would use them only if it’s impossible to set up the collocation 
> between 2 entities. That’s not your case from what I see. NON collocated 
> joins are slower than collocated ones.
>  
> —
> Denis
>  
>  
> On Sep 1, 2017, at 2:53 PM, Roger Fischer (CW) <rfis...@brocade.com 
> <mailto:rfis...@brocade.com>> wrote:
>  
> Hi Matt,
> 
> are the objects to join collocated, ie. do they have the same affinity key? 
> If yes, it should work (it worked for me).
> 
> If no, you need to enable distributed joins for the query. See the middle 
> line.
> 
>            SqlFieldsQuery qry = new SqlFieldsQuery( stmt);
>            qry.setDistributedJoins( true);
>            queryCursor = aCache.query( qry);
> 
> Roger
> 
> -----Original Message-----
> From: matt [mailto:goodie...@gmail.com <mailto:goodie...@gmail.com>] 
> Sent: Friday, September 01, 2017 1:52 PM
> To: user@ignite.apache.org <mailto:user@ignite.apache.org>
> Subject: SQLQuery with simple Join return no results
> 
> I have 2 caches defined, both with String keys, and classes that make use of 
> the Ignite annotations for indexes and affinity. I've got 3 different nodes 
> running, and the code I'm using to populate the cache w/test data works, and 
> I can see each node is updated with its share of the data. My index types are 
> set on the caches as well.
> 
> If I do a ScanQuery, I can see that all of the fields and IDs are correct, 
> Ignite returns them all. But when doing a SqlQuery, I get nothing back.
> Ignite is not complaining about the query, it's just returning an empty 
> cursor.
> 
> If I remove the Join, results are returned.
> 
> So I'm wondering if this is related to the way I've set up my affinity 
> mapping. It's basically setup like the code below... and the query looks like 
> this:
> 
> "from B, A WHERE B.id = A.bID"
> 
> Any ideas on what I'm doing wrong here?
> 
> class A implements Serializable {
>  @QuerySqlField(index = true)
>  String id;
> 
>  @QuerySqlField(index = true)
>  String bId;  
> 
>  @AffinityKeyMapped
>  @QuerySqlField(index = true)
>  String group;
> }
> 
> class B implements Serializable {
>  @QuerySqlField(index = true)
>  String id;
> 
>  @AffinityKeyMapped
>  @QuerySqlField(index = true)
>  String group;
> }
> 
> 
> 
> --
> Sent from: 
> https://urldefense.proofpoint.com/v2/url?u=http-3A__apache-2Dignite-2Dusers.70518.x6.nabble.com_&d=DwICAg&c=IL_XqQWOjubgfqINi2jTzg&r=1esZO0r0bYS90lcsaLA6N4AFxuNo6lzauhETGwdJQoQ&m=g9B7o3WZd7LuW39MRFWdU5Gim1S3DRPZGcugI0n9Wno&s=ehBogkdblpG8U9N0taPx5Jdf8G9uDMnZiRR-f34rEe0&e=
>  
> <https://urldefense.proofpoint.com/v2/url?u=http-3A__apache-2Dignite-2Dusers.70518.x6.nabble.com_&d=DwICAg&c=IL_XqQWOjubgfqINi2jTzg&r=1esZO0r0bYS90lcsaLA6N4AFxuNo6lzauhETGwdJQoQ&m=g9B7o3WZd7LuW39MRFWdU5Gim1S3DRPZGcugI0n9Wno&s=ehBogkdblpG8U9N0taPx5Jdf8G9uDMnZiRR-f34rEe0&e=>

Reply via email to