Hi Flavio,

yes, I think it's possible. I have one question before I try to explain
how:
do you model "Rome", "Berlin", "101.3" etc. in your above example as
separate vertices or as properties of vertices?

On 8 July 2015 at 10:43, Flavio Pompermaier <pomperma...@okkam.it> wrote:

> Let's say I have some nodes of some type of interest (type1 and type2).
> My source data looks like <sourceNodeId, type, edgeName, destNodeId>.
> For example, I could be interested in sourceNodes having type == Person
> to gather the value obtained from the expansion of some paths (for
> example livesIn.name and marriedTo.name). Notice that I could define other
> paths of interest for different types.. (For nodes of type Place I could be
> interested in gathering containedIn.name).
> If my source data looks like:
>
> (1, Person, livesIn, 2)
> (1, Person, livesIn, 3)
> (2, Place, name, "Rome")
> (2, Place, lat, 101.3)
> (2, Place, long, 101.3)
> (3, Place, name, "Berlin")
> (3, Place, containedIn, 4)
> (4, Country, name, "Germany")
>
> I'd like that node 1 (in the end) collect the following paths:
>
> livesIn.name : "Rome" (from node 2)
> livesIn.name : "Berlin" (from node 3)
> livesIn.containedIn.name: "Germany" (from node 4)
> marriedTo.name : null because not married :)
>
> So, in my vertexCentricIteration each Vertex knows its neighbors (e.g.
> node 1 knows that 2 and 3 should be queried for their "name" attribute).
> If they receive a message asking for "name" from node 1 they have to reply
> to node 1 with the value of that property.
>
> So in my implementation, I check whether my node has to send some query to
> neighbors and wait for the response. The problem is that node 3 for
> example, once queried for property containedIn.name from node 1 it just
> have to forward this path to node 4 and thell to 4 to reply to 1.
>
> Is that possible?
>
>
> On Wed, Jul 8, 2015 at 10:19 AM, Vasiliki Kalavri <
> vasilikikala...@gmail.com> wrote:
>
>> Is it the same message that you propagate or is it different for each
>> vertex / neighbor? If you have to store a <neighborID, msg> pair for each
>> neighbor, then you will quickly run out of memory. If it's the same message
>> you need to send, or you simply need to add the current vertex Id, then you
>> can probably get rid of the neighborID.
>>
>> By "outbox" I believe you mean storing them in the vertex value, correct?
>> I don't think you will have to explicitly reset it, as in each superstep
>> vertices only receive messages sent in the previous superstep, i.e. "old"
>> messages don't get re-sent.
>> On Jul 8, 2015 9:48 AM, "Flavio Pompermaier" <pomperma...@okkam.it>
>> wrote:
>>
>>> The problem is that my nodes have to gather data coming from some path
>>> of interest along the graph (depending on the type of the node), otherwise
>>> they just have to forward the received message adding their id to the
>>> message path (more or less). It's like a postal tracking system.
>>>
>>> The problem is that I have to reset the "outbox" of each vertex once the
>>> messages have been sent..
>>> Do you think that it makes sense in this case to have an outbox of
>>> messages (destination, message) in each vertex and reset it in the
>>> postSuperstep() of the VertexUpdateFunction?
>>>
>>> On Wed, Jul 8, 2015 at 9:38 AM, Vasiliki Kalavri <
>>> vasilikikala...@gmail.com> wrote:
>>>
>>>> Hi Flavio!
>>>>
>>>> Are you talking about vertex-centric iterations in gelly?
>>>>
>>>> If yes, you can send messages to a particular vertex with
>>>> "sendMessageTo(vertexId, msg)" and
>>>> to all neighbors with "sendMessageToAllNeighbors(msg)". These methods
>>>> are available inside the MessagingFunction.
>>>> Accessing received messages happens inside the VertexUpdateFunction.
>>>> So, the usual way of writing these programs is to:
>>>> (1) go through received messages in the VertexUpdateFunction and
>>>> compute the new vertex value based them
>>>> (2) compute and send messages in the MessagingFunction.
>>>>
>>>> Would that work in you case?
>>>>
>>>> Cheers,
>>>> Vasia.
>>>>
>>>> On 8 July 2015 at 08:47, Flavio Pompermaier <pomperma...@okkam.it>
>>>> wrote:
>>>>
>>>>>
>>>>> Hi to all,
>>>>> is there a way in gelly to forward received messages (and modify their
>>>>> content before sending)?
>>>>>
>>>>> Best,
>>>>> Flavio
>>>>>
>>>>>
>>>>
>>>
>

Reply via email to