You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@beam.apache.org by "Alexey Romanenko (Jira)" <ji...@apache.org> on 2021/03/17 18:27:00 UTC

[jira] [Comment Edited] (BEAM-11815) Fail to read more than 1M of items

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

Alexey Romanenko edited comment on BEAM-11815 at 3/17/21, 6:26 PM:
-------------------------------------------------------------------

Good catch, [~Safari]!

{code} 
If DynamoDB processes the number of items up to the limit while processing the results, it stops the operation and returns the matching values up to that point, and a key in LastEvaluatedKey to apply in a subsequent operation, so that you can pick up where you left off. Also, if the processed dataset size exceeds 1 MB before DynamoDB reaches this limit, it stops the operation and returns the matching values up to the limit, and a key in LastEvaluatedKey to apply in a subsequent operation to continue the operation.
{code}

So, the fix looks pretty simple on the first sight - we need to check the value of returned {{LastEvaluatedKey}} in the loop until everything was read. Though, I'm wondering if we need (and we can) to split it in advance to read in parallel despite the the Read is already split by {{SegmentId}} .

[~iemejia] I think we have a similar code for both versions of SDK and the fix will be similar too.


was (Author: aromanenko):
Good catch, [~Safari]!

{code} 
If DynamoDB processes the number of items up to the limit while processing the results, it stops the operation and returns the matching values up to that point, and a key in LastEvaluatedKey to apply in a subsequent operation, so that you can pick up where you left off. Also, if the processed dataset size exceeds 1 MB before DynamoDB reaches this limit, it stops the operation and returns the matching values up to the limit, and a key in LastEvaluatedKey to apply in a subsequent operation to continue the operation.
{code}

So, the fix looks pretty simple on the first sight - we need to check the value of returned {{LastEvaluatedKey}} in the loop until everything was read. Though, I'm wondering if we need (and we can) to split it in advance to read in parallel despite the the Read is already split by {{SegmentId}} .

> Fail to read more than 1M of items
> ----------------------------------
>
>                 Key: BEAM-11815
>                 URL: https://issues.apache.org/jira/browse/BEAM-11815
>             Project: Beam
>          Issue Type: Bug
>          Components: io-java-aws
>            Reporter: Mohammad
>            Assignee: Alexey Romanenko
>            Priority: P1
>
> In processElement:
> {code:java}
> private static class ReadFn<T> extends DoFn<Read<T>, T> {
>   @ProcessElement
>   public void processElement(@Element Read<T> spec, OutputReceiver<T> out) {
>     AmazonDynamoDB client = spec.getAwsClientsProvider().createDynamoDB();
>     ScanRequest scanRequest = spec.getScanRequestFn().apply(null);
>     scanRequest.setSegment(spec.getSegmentId());
>     ScanResult scanResult = client.scan(scanRequest);
>     out.output(spec.getScanResultMapperFn().apply(scanResult));
>   }
> }
> {code}
>  
>  
> if total requested items > 1M dynamodb will only send upto 1M of data  and expects client to read remaining items through LastEvaluatedKey (see [this|https://docs.aws.amazon.com/AWSJavaSDK/latest/javadoc/com/amazonaws/services/dynamodbv2/model/ScanRequest.html#setLimit-java.lang.Integer-])
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)