Hi Anton,
yes, if the keyCoder doesn't have proper hashCode and equals, then it
would manifest exactly as described.
Jan
On 5/9/19 6:28 PM, Anton Kedin wrote:
Does it look similar to
https://issues.apache.org/jira/browse/BEAM-6813 ? I also stumbled on a
problem with a state in DirectRunner but wasn't able to figure it out
yet:
https://lists.apache.org/thread.html/dae8b605a218532c085a0eea4e71338eae51922c26820f37b24875c0@%3Cdev.beam.apache.org%3E
Regards,
Anton
*From: *Jan Lukavský <je...@seznam.cz <mailto:je...@seznam.cz>>
*Date: *Thu, May 9, 2019 at 8:13 AM
*To: * <dev@beam.apache.org <mailto:dev@beam.apache.org>>
Because of the use of hashCode in StateSpecs, I'd say that it is.
But it is not obvious. That's why I'd suggest to make it abstract
on Coder, so that all implementations have to override it. That's
a simple solution, but the question is - should hashCode of Coder
be used that way? I think that StateSpec instances should be equal
only to itself. Then the hashCode can be stored in the instance, e.g.
private final int hashCode = System.identityHashCode(this)
and returned in hashCode(). There would be no need for Coder to
implement hashCode anymore (if there aren't any other cases, where
it is needed, in which case it would still be better to add
abstract hashCode and equals methods on Coder).
Jan
On 5/9/19 5:04 PM, Reuven Lax wrote:
Is a valid hashCode on Coder part of our contract or not? If it
is, then the lack of hashCode on SchemaCoder is simply a bug.
On Thu, May 9, 2019 at 7:42 AM Jan Lukavský <je...@seznam.cz
<mailto:je...@seznam.cz>> wrote:
Hi,
I have spent several hour digging into strange issue with
DirectRunner,
that manifested as non-deterministic run of pipeline. The
pipeline
contains basically only single stateful ParDo, which adds
elements into
state and after some timeout flushes these elements into
output. The
issues was, that sometimes (very often) when the timer fired,
the state
appeared to be empty, although I actually added something
into the
state. I will skip details, but the problem boils down to the
fact, that
StateSpecs hash Coder into hashCode - e.g.
@Override
public int hashCode() {
return Objects.hash(getClass(), coder);
}
in ValueStateSpec. Now, when Coder doesn't have hashCode and
equals
implemented (and there are some of those in the codebase
itself - e.g.
SchemaCoder), it all blows up in a very hard-to-debug manner.
So the
proposal is - either to add abstract hashCode and equals to
Coder, or
don't hash the Coder into hashCode of StateSpecs (we can
generate unique
ID for each StateSpec instance for example).
Any thoughts about which path to follow? Or maybe both? :)
Jan