t;
>
>
> On Mon, Apr 24, 2023 at 10:39 AM Murphy, Sean P.
> wrote:
>
> Any other thoughts? I’ve run out of ideas. Thanks, ~Sean
>
>
>
> *From: *Murphy, Sean P.
> *Date: *Friday, April 21, 2023 at 11:00 AM
> *To: *Alexey Romanenko , Evan Galpin <
&g
Date: Thursday, April 20, 2023 at 11:13 AM
To: user mailto:user@beam.apache.org>>, Murphy, Sean P.
mailto:murphy.s...@mayo.edu>>
Subject: [EXTERNAL] Re: Q: Apache Beam IOElasticsearchIO.read() method (Java),
which expects a PBegin input and a means to handle a collection of queries
Some
es ,
> egal...@apache.org
> *Subject: *Re: [EXTERNAL] Re: Q: Apache Beam IOElasticsearchIO.read()
> method (Java), which expects a PBegin input and a means to handle a
> collection of queries
>
> Thank you, Alexey.
>
>
>
> The issue isn’t with the split itself, but how to intr
For more info on splitable DoFn, there is a good resource on the beam
blog[1]. Alexey has also shown a great alternative!
[1] https://beam.apache.org/blog/splittable-do-fn/
On Thu, Apr 20, 2023 at 9:08 AM Alexey Romanenko
wrote:
> Some Java IO-connectors implement a class something like "class
Some Java IO-connectors implement a class something like "class ReadAll extends
PTransform, PCollection>” where “Read” is
supposed to be configured dynamically. As a simple example, take a look on
“SolrIO” [1]
So, to support what you are looking for, “ReadAll”-pattern should be
implemented fo
@beam.apache.org
Cc: Murphy, Sean P.
Subject: [EXTERNAL] Re: Q: Apache Beam IOElasticsearchIO.read() method (Java),
which expects a PBegin input and a means to handle a collection of queries
Yes unfortunately the ES IO connector is not built in a way that can work by
taking inputs from a PCollection to
Yes unfortunately the ES IO connector is not built in a way that can work
by taking inputs from a PCollection to issue Reads. The most scalable way
to support this is to revisit the implementation of Elasticsearch Read
transform and instead implement it as a SplittableDoFn.
On Wed, Apr 19, 2023 at
Hi Sean,
I'm not an expert but I think the .withQuery() functions takes part of the
build stage rather than the runtime stage.
This means that the way ElasticsearchIO was built is so that while the
pipeline is being built you could set the query but it is not possible
during runtime which mean you
I'm running into an issue using the ElasticsearchIO.read() to handle more than
one instance of a query. My queries are being dynamically built as a
PCollection based on an incoming group of values. I'm trying to see how to load
the .withQuery() parameter which could provide this capability or an