[ 
https://issues.apache.org/jira/browse/BEAM-10945?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17478416#comment-17478416
 ] 

Etienne Chauchot commented on BEAM-10945:
-----------------------------------------

[~kenn] [~DraCzech] indeed this setup was never tested because it did not exist 
at the time the IO was made. That being said we must protect against its 
particularities of not being able to estimate the size of an index (I guess 
because the index is spread across several clusters with their own stats). 
[~kenn] I bet it is still happening  as there were no change in the size 
estimation in production code for ages. [~DraCzech] if you'd like to 
contribute, you can submit whatever is needed to impl these protections. If you 
need to submit to both DR and ESIO it needs to be 2 separate PRs with links to 
each other. I could then review the ES part with [~egalpin] if needed. You can 
test on flink but DR is the reference runner for tests so it will need to be 
changed if it lacks defensive code. [~kenn]can you give [~DraCzech] contributor 
role to assign the ticket ?

> ElasticsearchIO performs 0 division on DirectRunner
> ---------------------------------------------------
>
>                 Key: BEAM-10945
>                 URL: https://issues.apache.org/jira/browse/BEAM-10945
>             Project: Beam
>          Issue Type: Bug
>          Components: io-java-elasticsearch, runner-direct
>    Affects Versions: 2.23.0
>         Environment: Beam 2.23
> Java 1.8.0_265
> Ubuntu 16.04
> Elastic version of cluster 7.9.1, cross cluster setup
> Parallelism of direct runner 8
>            Reporter: Milan Nikl
>            Priority: P3
>
> h1. Environment configuration
> In my company we use [Elasticsearch cross 
> cluster|https://www.elastic.co/guide/en/elasticsearch/reference/current/modules-cross-cluster-search.html#ccs-supported-apis]
>  setup for search. Cluster version is 7.9.1.
> I intended to use ElasticsearchIO for reading application logs and 
> subsequently producing some aggregated data.
> h1. Problem description
>  # In cross cluster ES setup, there is no {{/<index>/_stats}} API available, 
> so it is not possible to compute 
> [ElasticsearchIO#getEstimatedSizeBytes|https://github.com/apache/beam/blob/master/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java#L692]
>  properly.
>  # {{statsJson}} returned by the cluster looks like this:
> {quote}Unknown macro: \{ "_shards" }
>  ,
>  "_all" :
>  Unknown macro: \{ "primaries" }
>  ,
>  "total" : \{ }
>  },
>  "indices" : \{ }
>  }
> {quote}
>  # That means that {{totalCount}} value cannot be parsed from the json and is 
> thus set to {{0}}.
>  # Which means that {{estimatedByteSize}} value [is set to 
> 1|https://github.com/apache/beam/blob/master/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java#L707]
>  (Which itself is a workaround for similar issue.)
>  # {{ElasticsearchIO#getEstimatedSizeBytes}} is used in 
> [BoundedReadEvaluatorFactory#getInitialInputs|https://github.com/apache/beam/blob/master/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java#L212]
>  which does not check the value and performs division of two {{long}} values, 
> which of course results in {{0}} for any {{targetParallelism > 1}}.
>  # Then 
> [ElasticsearchIO#split|https://github.com/apache/beam/blob/master/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java#L665]
>  is called with {{indexSize = 1}} and {{desiredBundleSizeBytes = 1}}. Which 
> sets {{nbBundlesFloat}} value to infinity.
>  # Even though the number of bundles is ceiled at {{1024}}, reading from 1024 
> BoundedElasticsearchSources concurrently makes the ElasticsearchIO virtually 
> impossible to use on direct runner.
> h1. Resolution suggestion
> I still haven't tested reading from ElasticsearchIO on proper runner (we use 
> flink 1.10.2), so I cannot either confirm or deny its functionality on our 
> elastic setup. At the moment I'm just suggesting few checks of input values 
> so the zero division and unnecessary parallelism problems are eliminated on 
> direct runner.



--
This message was sent by Atlassian Jira
(v8.20.1#820001)

Reply via email to