flink-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Fabian Hueske <fhue...@gmail.com>
Subject Re: Gelly vertex ID type requirements?
Date Thu, 30 Jul 2015 22:29:57 GMT
Hi,

I opened a JIRA (FLINK-2442) and submitted a PR (#963) for the "Wrong field
type" problem.
Is the other problem is addressed in FLINK-2437?

Cheers, Fabian

2015-07-30 16:29 GMT+02:00 Gábor Gévay <ggab90@gmail.com>:

> Thanks for the response.
> As a temporary workaround, I tried to change these problematic lines:
>
> } else {
>    Preconditions.checkArgument(fieldType instanceof AtomicType, "Wrong
> field type: " + fieldType.toString());
>    keyFields.add(new FlatFieldDescriptor(keyId, fieldType));
> }
>
> into this:
>
> } else if (fieldType instanceof AtomicType) {
>    keyFields.add(new FlatFieldDescriptor(keyId, fieldType));
> } else {
>    Preconditions.checkArgument(fieldType instanceof PojoTypeInfo,
> "Wrong field type: " + fieldType.toString());
>    ((PojoTypeInfo)fieldType).getFlatFields("*", 0, keyFields);
> }
>
>
> But then I ran into another problem: The TypeExtractor creates the
> TupleTypeInfoBase for the Edge type of my graph with the following
> types:
>
> 0 = {PojoTypeInfo@1067} "PojoType<malom.GameState, fields = [board:
> Long, sid: PojoType<malom.SectorId, fields = [b: Byte, bf: Byte, w:
> Byte, wf: Byte]>]>"
> 1 = {GenericTypeInfo@1068} "GenericType<malom.GameState>"
> 2 = {ValueTypeInfo@1069} "ValueType<NullValue>"
>
> The problem here is that the first two types should clearly be the
> same, as the Edge type looks like this:
> public class Edge<K, V> extends Tuple3<K, K, V>
>
> I did a bit of debugging on this, and the source of the problem seems
> to be the mechanism in TypeExtractor that would detect recursive types
> (see the "alreadySeen" field in TypeExtractor), as it mistakes the
> second appearance of malom.GameState with a recursive field.
>
> Specifically the following happens: createTypeInfoWithTypeHierarchy
> starts to process the Edge type, and in line 433 it calls itself for
> the first field, which proceeds into the privateGetForClass case which
> correctly detects that it is a POJO, and correctly returns a
> PojoTypeInfo; but in the meantime in line 1190, privateGetForClass
> adds malom.GameState to "alreadySeen". Then the outer
> createTypeInfoWithTypeHierarchy approaches the second field, goes into
> privateGetForClass, which mistakenly returns a GenericTypeInfo, as it
> thinks in line 1186, that a recursive type is being processed.
>
> Should I open a Jira for this?
>
> A possible solution would be to change the alreadySeen field into a
> parameter of all the various type extraction methods, and make it
> contain only those types that are ancestors in the nesting hierarchy.
>
> Best regards,
> Gabor
>
> 2015-07-30 14:32 GMT+02:00 Fabian Hueske <fhueske@gmail.com>:
> > Thanks for reporting this issue.
> > The "Wrong field type" error looks like a bug to me.
> > This happens, because PojoType is neither a TupleType nor an AtomicType.
> To
> > me it looks like the TupleTypeInfoBase condition should be generalized to
> > CompositeType.
> >
> > I will look into this.
> >
> > Cheers, Fabian
> >
> > 2015-07-30 14:18 GMT+02:00 Gábor Gévay <ggab90@gmail.com>:
> >>
> >> Hello,
> >>
> >> I am having some trouble building a graph where the vertex ID type is
> >> a POJO. Specifically, it would be a class that has two fields: a long
> >> field, and a field which is of another class that has four byte
> >> fields. (Both classes implement the Comparable interface, as the Gelly
> >> guide specifies.) If the outer class has a default constructor, then
> >> it is recognized as a POJO, but I get the following exception:
> >>
> >> Exception in thread "main" java.lang.IllegalArgumentException: Wrong
> >> field type: PojoType<malom.GameState, fields = [board: Long, sid:
> >> PojoType<malom.SectorId, fields = [b: Byte, bf: Byte, w: Byte, wf:
> >> Byte]>]>
> >>     at
> >>
> com.google.common.base.Preconditions.checkArgument(Preconditions.java:122)
> >>     at
> >>
> org.apache.flink.api.java.operators.Keys$ExpressionKeys.<init>(Keys.java:241)
> >>     at
> >>
> org.apache.flink.api.java.operators.Keys$ExpressionKeys.<init>(Keys.java:203)
> >>     at
> >>
> org.apache.flink.api.java.operators.CoGroupOperator$CoGroupOperatorSets.where(CoGroupOperator.java:458)
> >>     at org.apache.flink.graph.Graph.inDegrees(Graph.java:701)
> >>     at
> >>
> org.apache.flink.graph.spargel.VertexCentricIteration.createResultVerticesWithDegrees(VertexCentricIteration.java:610)
> >>     at
> >>
> org.apache.flink.graph.spargel.VertexCentricIteration.createResult(VertexCentricIteration.java:180)
> >>     at org.apache.flink.api.java.DataSet.runOperation(DataSet.java:1044)
> >>     at
> >> org.apache.flink.graph.Graph.runVertexCentricIteration(Graph.java:1312)
> >>     at malom.Retrograde.run(Retrograde.java:64)
> >>     at malom.Solver.main(Solver.java:32)
> >>     at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> >>     at
> >>
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
> >>     at
> >>
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> >>     at java.lang.reflect.Method.invoke(Method.java:497)
> >>     at
> >> com.intellij.rt.execution.application.AppMain.main(AppMain.java:140)
> >>
> >> Note, that originally the exception just said "Wrong field type", from
> >> which I had no idea what type is it referring to, so I modified line
> >> 241 of Keys.java to include the type in the msg like this:
> >> Preconditions.checkArgument(fieldType instanceof AtomicType, "Wrong
> >> field type: " + fieldType.toString());
> >>
> >> On the other hand, if I comment out the default constructor of the
> >> outer class, then it is not a POJO, but only a GenericTypeInfo is
> >> created from it, which implements AtomicType, so the previous
> >> exception does not appear. Does this mean that the Vertex IDs cannot
> >> be POJOs? I am not sure if this is the intended behaviour. After all,
> >> if we have a POJO, that should always be better then if we have a
> >> generic type, right?
> >>
> >> (I am just guessing here, but maybe CompositeType could also be
> >> regarded as an AtomicType: the only method declared by the AtomicType
> >> interface is createComparator, which is also defined in CompositeType
> >> (of which PojoTypeInfo is a subclass), but with different parameters,
> >> but maybe CompositeType could implement AtomicType by delegating the
> >> createComparator call with all of the fields specified?)
> >>
> >> I encountered another problem, which is may or may not be related to
> >> the above: without the default constructor (the GenericTypeInfo case),
> >> the VertexCentricIteration "eats" my graph, that is, the result graph
> >> has zero vertices. I traced this problem to the first join in
> >> VertexCentricIteration.createResultVerticesWithDegrees, where the
> >> "degrees" DataSet is created: both inputs of the join (outDegrees and
> >> inDegrees) contains the correct data, but the result (degrees) is
> >> empty. Interestingly, this problem disappears, if I add
> >> JoinHint.REPARTITION_SORT_MERGE.
> >>
> >> Best regards,
> >> Gabor
> >
> >
>

Mime
View raw message