Hi Lukasz,
I've created JIRA issue [1] and PR [2].
Jan
[1] https://issues.apache.org/jira/browse/BEAM-7269
[2] https://github.com/apache/beam/pull/8555
On 5/10/19 7:39 PM, Lukasz Cwik wrote:
That seems like the correct fix as well. We could open up a PR and see
what the tests catch as a first pass for understanding the implications.
On Fri, May 10, 2019 at 9:31 AM Jan Lukavský <je...@seznam.cz
<mailto:je...@seznam.cz>> wrote:
Hm, yes, the fix might be also in fixing hashCode and equals of
SimpleStateTag, so that it doesn't hash and compare the StateSpec,
but only the StructureId. That looks like best option to me. But
I'm not sure about other implications this might have.
Jan
On 5/10/19 5:43 PM, Reuven Lax wrote:
Ok so this sounds like a bug in the DirectRunner then?
*From: *Lukasz Cwik <lc...@google.com <mailto:lc...@google.com>>
*Date: *Fri, May 10, 2019 at 8:38 AM
*To: *dev
StateSpec should not be used as a key within any maps. We
should use the logical name of the StateSpec relative to the
DoFn as its id and should only be using that id for
comparisons/lookups.
On Fri, May 10, 2019 at 1:07 AM Jan Lukavský <je...@seznam.cz
<mailto:je...@seznam.cz>> wrote:
I'm not sure. Generally it affects any runner that uses
HashMap to store StateSpec.
Jan
On 5/9/19 6:32 PM, Reuven Lax wrote:
Is this specific to the DirectRunner, or does it affect
other runners?
On Thu, May 9, 2019 at 8:13 AM Jan Lukavský
<je...@seznam.cz <mailto:je...@seznam.cz>> wrote:
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