You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@beam.apache.org by Arunkumar Santhanagopalan <ar...@gmail.com> on 2017/09/13 03:39:42 UTC
Migration From 1.9.x to 2.1.0
Hi,
We are trying to migrate from Dataflow 1.9.x to Dataflow 2.1.0
I need help with the following changes
1.
class Join extends DoFn<String, String> {
@Override
public void startBundle(Context c) throws Exception {
super.startBundle(c);
createParser();
}
Method "startBundle" does not override method startBundle from its
superclass
2.
class Join extends DoFn<String, String>{
private final Aggregator<Long, Long> duplicatesCount =
createAggregator(DUPLICATES_COUNTER, new Sum.SumLongFn());
cannot resolve method 'createAggregator, Sum.SumLongFn has a private access
3.
class Join extends DoFn<String, String>{
public void processElement(ProcessContext c) {
c.sideOutput(duplicatesTag, s)
}
cannot resolve method sideOutput(org.apache.beam.sdk.values.TupleTag)
4.
public abstract class ReadCsv<T> extends DoFn<String, T> {
@Override
public final void processElement(ProcessContext c) throws Exception {
T output = processElement(c.element(), c);
if (output != null) {
c.output(output);
}
}
Method does not override method processElement from its superclass
5.
import org.apache.beam.sdk.io.TextIO;
TextIO.Read.from("gs://spins/data/part-*")
Non-static method "from" cannot be referenced from static context
Re: Migration From 1.9.x to 2.1.0
Posted by Eugene Kirpichov <ki...@google.com.INVALID>.
The full set of changes is described in
https://cloud.google.com/dataflow/release-notes/release-notes-java-2
On Wed, Sep 13, 2017 at 8:53 AM Thomas Groh <tg...@google.com.invalid>
wrote:
> for (1) and (4), the DoFn methods have been moved to be reflection based.
> Instead of using `@Override` in your DoFns, you should annotate those
> methods with `@StartBundle`, `@ProcessElement`, and `@FinishBundle`
> instead.
>
> For (2), Aggregators have been removed. Our suggested replacement is the
> use of the `Metrics` class - in this case, a Counter metric is appropriate.
>
> For (3), `sideOutput` has been renamed to `output`; the use is otherwise
> identical.
>
> for (5), the pattern has changed from `TextIO.Read.from(...)` to
> `TextIO.read().from(...)` (which should allow the remainder of the
> PTransform to also be configured without having to specify a Filepattern up
> front)
>
> On Tue, Sep 12, 2017 at 8:39 PM, Arunkumar Santhanagopalan <
> arunk777@gmail.com> wrote:
>
> > Hi,
> >
> > We are trying to migrate from Dataflow 1.9.x to Dataflow 2.1.0
> >
> > I need help with the following changes
> >
> >
> > 1.
> > class Join extends DoFn<String, String> {
> > @Override
> > public void startBundle(Context c) throws Exception {
> > super.startBundle(c);
> > createParser();
> > }
> >
> > Method "startBundle" does not override method startBundle from its
> > superclass
> >
> >
> > 2.
> > class Join extends DoFn<String, String>{
> > private final Aggregator<Long, Long> duplicatesCount =
> > createAggregator(DUPLICATES_COUNTER, new Sum.SumLongFn());
> >
> > cannot resolve method 'createAggregator, Sum.SumLongFn has a private
> access
> >
> > 3.
> > class Join extends DoFn<String, String>{
> > public void processElement(ProcessContext c) {
> > c.sideOutput(duplicatesTag, s)
> > }
> > cannot resolve method sideOutput(org.apache.beam.sdk.values.TupleTag)
> >
> >
> > 4.
> > public abstract class ReadCsv<T> extends DoFn<String, T> {
> >
> > @Override
> > public final void processElement(ProcessContext c) throws Exception {
> > T output = processElement(c.element(), c);
> > if (output != null) {
> > c.output(output);
> > }
> > }
> > Method does not override method processElement from its superclass
> >
> >
> > 5.
> > import org.apache.beam.sdk.io.TextIO;
> >
> > TextIO.Read.from("gs://spins/data/part-*")
> > Non-static method "from" cannot be referenced from static context
> >
>
Re: Migration From 1.9.x to 2.1.0
Posted by Thomas Groh <tg...@google.com.INVALID>.
for (1) and (4), the DoFn methods have been moved to be reflection based.
Instead of using `@Override` in your DoFns, you should annotate those
methods with `@StartBundle`, `@ProcessElement`, and `@FinishBundle` instead.
For (2), Aggregators have been removed. Our suggested replacement is the
use of the `Metrics` class - in this case, a Counter metric is appropriate.
For (3), `sideOutput` has been renamed to `output`; the use is otherwise
identical.
for (5), the pattern has changed from `TextIO.Read.from(...)` to
`TextIO.read().from(...)` (which should allow the remainder of the
PTransform to also be configured without having to specify a Filepattern up
front)
On Tue, Sep 12, 2017 at 8:39 PM, Arunkumar Santhanagopalan <
arunk777@gmail.com> wrote:
> Hi,
>
> We are trying to migrate from Dataflow 1.9.x to Dataflow 2.1.0
>
> I need help with the following changes
>
>
> 1.
> class Join extends DoFn<String, String> {
> @Override
> public void startBundle(Context c) throws Exception {
> super.startBundle(c);
> createParser();
> }
>
> Method "startBundle" does not override method startBundle from its
> superclass
>
>
> 2.
> class Join extends DoFn<String, String>{
> private final Aggregator<Long, Long> duplicatesCount =
> createAggregator(DUPLICATES_COUNTER, new Sum.SumLongFn());
>
> cannot resolve method 'createAggregator, Sum.SumLongFn has a private access
>
> 3.
> class Join extends DoFn<String, String>{
> public void processElement(ProcessContext c) {
> c.sideOutput(duplicatesTag, s)
> }
> cannot resolve method sideOutput(org.apache.beam.sdk.values.TupleTag)
>
>
> 4.
> public abstract class ReadCsv<T> extends DoFn<String, T> {
>
> @Override
> public final void processElement(ProcessContext c) throws Exception {
> T output = processElement(c.element(), c);
> if (output != null) {
> c.output(output);
> }
> }
> Method does not override method processElement from its superclass
>
>
> 5.
> import org.apache.beam.sdk.io.TextIO;
>
> TextIO.Read.from("gs://spins/data/part-*")
> Non-static method "from" cannot be referenced from static context
>