[ 
https://issues.apache.org/jira/browse/BEAM-3516?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Niel Markwick resolved BEAM-3516.
---------------------------------
    Resolution: Fixed

It is practically impossible for the spanner writer to calculate exactly how 
many cells would be modified by a single Mutation -- any indexes on the mutated 
rows will contribute to the number of modified cells, and the quantity of bytes 
stored.

Spanners limits are 20K mutated cells and 100Mb mutated bytes.

The SpannerIO writer's defaults are to estimate the size and batch up to 5K 
mutated cells and 1Mb of mutated data, which would not trigger this issue for 
most users.

[PR #4860|https://github.com/apache/beam/pull/4860]  and  [PR 
#5297|https://github.com/apache/beam/pull/5297] add functionality to estimate 
the number of cells and mutation size. 

[PR #6478|https://github.com/apache/beam/pull/6478] improves the documentation 
to mention the batching mechanism, when this error occurs and how to work 
around it.

> SpannerWriteGroupFn does not respect mutation limits
> ----------------------------------------------------
>
>                 Key: BEAM-3516
>                 URL: https://issues.apache.org/jira/browse/BEAM-3516
>             Project: Beam
>          Issue Type: Bug
>          Components: runner-dataflow
>    Affects Versions: 2.2.0
>            Reporter: Ryan Gordon
>            Assignee: Niel Markwick
>            Priority: Major
>             Fix For: 2.9.0
>
>          Time Spent: 5h 20m
>  Remaining Estimate: 0h
>
> When using SpannerIO.write(), if it happens to be a large batch or a table 
> with indexes its very possible it can hit the Spanner Mutations Limitation 
> and fail with the following error:
> {quote}Jan 02, 2018 2:42:59 PM 
> org.apache.beam.runners.dataflow.util.MonitoringUtil$LoggingHandler process
> SEVERE: 2018-01-02T22:42:57.873Z: (3e7c871d215e890b): 
> com.google.cloud.spanner.SpannerException: INVALID_ARGUMENT: 
> io.grpc.StatusRuntimeException: INVALID_ARGUMENT: The transaction contains 
> too many mutations. Insert and update operations count with the multiplicity 
> of the number of columns they affect. For example, inserting values into one 
> key column and four non-key columns count as five mutations total for the 
> insert. Delete and delete range operations count as one mutation regardless 
> of the number of columns affected. The total mutation count includes any 
> changes to indexes that the transaction generates. Please reduce the number 
> of writes, or use fewer indexes. (Maximum number: 20000)
> links {
>  description: "Cloud Spanner limits documentation."
>  url: "https://cloud.google.com/spanner/docs/limits";
> }
> at 
> com.google.cloud.spanner.SpannerExceptionFactory.newSpannerExceptionPreformatted(SpannerExceptionFactory.java:119)
>  at 
> com.google.cloud.spanner.SpannerExceptionFactory.newSpannerException(SpannerExceptionFactory.java:43)
>  at 
> com.google.cloud.spanner.SpannerExceptionFactory.newSpannerException(SpannerExceptionFactory.java:80)
>  at 
> com.google.cloud.spanner.spi.v1.GrpcSpannerRpc.get(GrpcSpannerRpc.java:404)
>  at 
> com.google.cloud.spanner.spi.v1.GrpcSpannerRpc.commit(GrpcSpannerRpc.java:376)
>  at 
> com.google.cloud.spanner.SpannerImpl$SessionImpl$2.call(SpannerImpl.java:729)
>  at 
> com.google.cloud.spanner.SpannerImpl$SessionImpl$2.call(SpannerImpl.java:726)
>  at com.google.cloud.spanner.SpannerImpl.runWithRetries(SpannerImpl.java:200)
>  at 
> com.google.cloud.spanner.SpannerImpl$SessionImpl.writeAtLeastOnce(SpannerImpl.java:725)
>  at 
> com.google.cloud.spanner.SessionPool$PooledSession.writeAtLeastOnce(SessionPool.java:248)
>  at 
> com.google.cloud.spanner.DatabaseClientImpl.writeAtLeastOnce(DatabaseClientImpl.java:37)
>  at 
> org.apache.beam.sdk.io.gcp.spanner.SpannerWriteGroupFn.flushBatch(SpannerWriteGroupFn.java:108)
>  at 
> org.apache.beam.sdk.io.gcp.spanner.SpannerWriteGroupFn.processElement(SpannerWriteGroupFn.java:79)
> {quote}
>  
> As a workaround we can override the "withBatchSizeBytes" to something much 
> smaller:
> {quote}mutations.apply("Write", SpannerIO
>    .write()
>    // Artificially reduce the max batch size b/c the batcher currently doesn't
>    // take into account the 20000 mutation multiplicity limit
>    .withBatchSizeBytes(1024) // 1KB
>    .withProjectId("#PROJECTID#")
>    .withInstanceId("#INSTANCE#")
>    .withDatabaseId("#DATABASE#")
>  );
> {quote}
> While this is not as efficient, it at least allows it to work consistently



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to