[BEAM-8157] Add LengthPrefixCoder around unknown key coder for stateful ProcessBundleDescriptor

Flink and other Runners use the serialized key in stateful ExecutableStages to
key partition the data. When state requests are issued from the SDK Harness the
key is also sent over serialized. However, the binary representation of that key
currently does not always match the Runner's representation. This causes
troubles with persisting keyed state for checkpoints.

When the key coder is known by the Runner, then it uses the same encoding scheme
as the SDK Harness. However, when the coder is unknown, it will be replaced by a
"LengthPrefixCoder<ByteArrayCoder>" which is problematic because the SDK Harness
does not add this coder and thus may produce a different encoding as the Runner.

The solution is to add a LengthPrefixCoder around unknown key coders in the
ProcessBundleDescriptor, such that the SDK Harness will length prefix the key
correctly.

Otherwise we run into this error:

```
Caused by: java.util.concurrent.ExecutionException: java.lang.RuntimeException: Error received from SDK harness for instruction 4: Traceback (most recent call last):
  File "/srv/venvs/service/trusty/service_venv/local/lib/python2.7/site-packages/apache_beam/runners/worker/sdk_worker.py", line 168, in _execute
    response = task()
  File "/srv/venvs/service/trusty/service_venv/local/lib/python2.7/site-packages/apache_beam/runners/worker/sdk_worker.py", line 201, in <lambda>
    self._execute(lambda: worker.do_instruction(work), work)
  File "/srv/venvs/service/trusty/service_venv/local/lib/python2.7/site-packages/apache_beam/runners/worker/sdk_worker.py", line 356, in do_instruction
    request.instruction_id)
  File "/srv/venvs/service/trusty/service_venv/local/lib/python2.7/site-packages/apache_beam/runners/worker/sdk_worker.py", line 382, in process_bundle
    bundle_processor.process_bundle(instruction_id))
  File "/srv/venvs/service/trusty/service_venv/local/lib/python2.7/site-packages/apache_beam/runners/worker/bundle_processor.py", line 667, in process_bundle
    data.ptransform_id].process_encoded(data.data)
  File "/srv/venvs/service/trusty/service_venv/local/lib/python2.7/site-packages/apache_beam/runners/worker/bundle_processor.py", line 143, in process_encoded
    self.output(decoded_value)
  File "apache_beam/runners/worker/operations.py", line 255, in apache_beam.runners.worker.operations.Operation.output
    def output(self, windowed_value, output_index=0):
  File "apache_beam/runners/worker/operations.py", line 256, in apache_beam.runners.worker.operations.Operation.output
    cython.cast(Receiver, self.receivers[output_index]).receive(windowed_value)
  File "apache_beam/runners/worker/operations.py", line 143, in apache_beam.runners.worker.operations.SingletonConsumerSet.receive
    self.consumer.process(windowed_value)
  File "apache_beam/runners/worker/operations.py", line 593, in apache_beam.runners.worker.operations.DoOperation.process
    with self.scoped_process_state:
  File "apache_beam/runners/worker/operations.py", line 594, in apache_beam.runners.worker.operations.DoOperation.process
    delayed_application = self.dofn_receiver.receive(o)
  File "apache_beam/runners/common.py", line 776, in apache_beam.runners.common.DoFnRunner.receive
    self.process(windowed_value)
  File "apache_beam/runners/common.py", line 782, in apache_beam.runners.common.DoFnRunner.process
    self._reraise_augmented(exn)
  File "apache_beam/runners/common.py", line 849, in apache_beam.runners.common.DoFnRunner._reraise_augmented
    raise_with_traceback(new_exn)
  File "apache_beam/runners/common.py", line 780, in apache_beam.runners.common.DoFnRunner.process
    return self.do_fn_invoker.invoke_process(windowed_value)
  File "apache_beam/runners/common.py", line 587, in apache_beam.runners.common.PerWindowInvoker.invoke_process
    self._invoke_process_per_window(
  File "apache_beam/runners/common.py", line 659, in apache_beam.runners.common.PerWindowInvoker._invoke_process_per_window
    output_processor.process_outputs(
  File "apache_beam/runners/common.py", line 880, in apache_beam.runners.common._OutputProcessor.process_outputs
    def process_outputs(self, windowed_input_element, results):
  File "apache_beam/runners/common.py", line 895, in apache_beam.runners.common._OutputProcessor.process_outputs
    for result in results:
  File "pricingrealtime/event_processing/stateful_event_processing.py", line 55, in process
    recent_events_map = StatefulEventDoFn._load_recent_events_map(recent_events_state)
  File "pricingrealtime/event_processing/stateful_event_processing.py", line 127, in _load_recent_events_map
    items_in_recent_events_bag = [e for e in recent_events_state.read()]
  File "/srv/venvs/service/trusty/service_venv/local/lib/python2.7/site-packages/apache_beam/runners/worker/bundle_processor.py", line 335, in __iter__
    for elem in self.first:
  File "/srv/venvs/service/trusty/service_venv/local/lib/python2.7/site-packages/apache_beam/runners/worker/sdk_worker.py", line 723, in _materialize_iter
    self._underlying.get_raw(state_key, continuation_token)
  File "/srv/venvs/service/trusty/service_venv/local/lib/python2.7/site-packages/apache_beam/runners/worker/sdk_worker.py", line 603, in get_raw
    continuation_token=continuation_token)))
  File "/srv/venvs/service/trusty/service_venv/local/lib/python2.7/site-packages/apache_beam/runners/worker/sdk_worker.py", line 637, in _blocking_request
    raise RuntimeError(response.error)
RuntimeError: java.lang.IllegalStateException: The current key '[1, -104, -97, -93, -34, -73, -128, -42, 36]' with key group index '274' does not belong to the key group range 'KeyGroupRange{startKeyGroup=153, endKeyGroup=154}'. Runner KeyCoder: LengthPrefixCoder(ByteArrayCoder). Ptransformid: ref_AppliedPTransform_process_events_with_stateful_dofn_23 Userstateid: recent_events
	at org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState(Preconditions.java:531)
	at org.apache.beam.runners.flink.translation.wrappers.streaming.ExecutableStageDoFnOperator$BagUserStateFactory$1.prepareStateBackend(ExecutableStageDoFnOperator.java:387)
	at org.apache.beam.runners.flink.translation.wrappers.streaming.ExecutableStageDoFnOperator$BagUserStateFactory$1.get(ExecutableStageDoFnOperator.java:309)
	at org.apache.beam.runners.flink.translation.wrappers.streaming.ExecutableStageDoFnOperator$BagUserStateFactory$1.get(ExecutableStageDoFnOperator.java:303)
	at org.apache.beam.runners.fnexecution.state.StateRequestHandlers$ByteStringStateRequestHandlerToBagUserStateHandlerFactoryAdapter.handleGetRequest(StateRequestHandlers.java:468)
	at org.apache.beam.runners.fnexecution.state.StateRequestHandlers$ByteStringStateRequestHandlerToBagUserStateHandlerFactoryAdapter.handle(StateRequestHandlers.java:415)
	at org.apache.beam.runners.fnexecution.state.StateRequestHandlers$StateKeyTypeDelegatingStateRequestHandler.handle(StateRequestHandlers.java:206)
	at org.apache.beam.runners.fnexecution.state.GrpcStateService$Inbound.onNext(GrpcStateService.java:130)
	at org.apache.beam.runners.fnexecution.state.GrpcStateService$Inbound.onNext(GrpcStateService.java:118)
	at org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.ServerCalls$StreamingServerCallHandler$StreamingServerCallListener.onMessage(ServerCalls.java:249)
	at org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ForwardingServerCallListener.onMessage(ForwardingServerCallListener.java:33)
	at org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Contexts$ContextualizedServerCallListener.onMessage(Contexts.java:76)
	at org.apache.beam.vendor.grpc.v1p21p0.io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.messagesAvailable(ServerCallImpl.java:297)
	at org.apache.beam.vendor.grpc.v1p21p0.io.grpc.internal.ServerImpl$JumpToApplicationThreadServerStreamListener$1MessagesAvailable.runInContext(ServerImpl.java:738)
	at org.apache.beam.vendor.grpc.v1p21p0.io.grpc.internal.ContextRunnable.run(ContextRunnable.java:37)
	at org.apache.beam.vendor.grpc.v1p21p0.io.grpc.internal.SerializingExecutor.run(SerializingExecutor.java:123)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
 [while running 'process_events_with_stateful_dofn']
```
3 files changed
tree: 30857dc5b4bbf04de0bbdce14900e2428c277084
  1. .github/
  2. .test-infra/
  3. buildSrc/
  4. examples/
  5. gradle/
  6. learning/
  7. model/
  8. ownership/
  9. release/
  10. runners/
  11. sdks/
  12. vendor/
  13. website/
  14. .gitattributes
  15. .gitignore
  16. .mailmap
  17. assembly.xml
  18. build.gradle
  19. CONTRIBUTING.md
  20. gradle.properties
  21. gradlew
  22. gradlew.bat
  23. LICENSE
  24. NOTICE
  25. OWNERS
  26. README.md
  27. settings.gradle
README.md

Apache Beam

Apache Beam is a unified model for defining both batch and streaming data-parallel processing pipelines, as well as a set of language-specific SDKs for constructing pipelines and Runners for executing them on distributed processing backends, including Apache Apex, Apache Flink, Apache Spark, Google Cloud Dataflow and Hazelcast Jet.

Status

Maven Version PyPI version Build Status Coverage Status Compat Check PyPI Compat Check at master

Post-commit tests status (on master branch)

LangSDKApexDataflowFlinkGearpumpSamzaSpark
GoBuild Status------Build Status------Build Status
JavaBuild StatusBuild StatusBuild StatusBuild Status
Build Status
Build Status
Build StatusBuild StatusBuild Status
Build Status
PythonBuild Status
Build Status
Build Status
Build Status
---Build Status
Build Status
Build Status
Build Status
------Build Status
XLang---------Build Status---------

Overview

Beam provides a general approach to expressing embarrassingly parallel data processing pipelines and supports three categories of users, each of which have relatively disparate backgrounds and needs.

  1. End Users: Writing pipelines with an existing SDK, running it on an existing runner. These users want to focus on writing their application logic and have everything else just work.
  2. SDK Writers: Developing a Beam SDK targeted at a specific user community (Java, Python, Scala, Go, R, graphical, etc). These users are language geeks, and would prefer to be shielded from all the details of various runners and their implementations.
  3. Runner Writers: Have an execution environment for distributed processing and would like to support programs written against the Beam Model. Would prefer to be shielded from details of multiple SDKs.

The Beam Model

The model behind Beam evolved from a number of internal Google data processing projects, including MapReduce, FlumeJava, and Millwheel. This model was originally known as the “Dataflow Model”.

To learn more about the Beam Model (though still under the original name of Dataflow), see the World Beyond Batch: Streaming 101 and Streaming 102 posts on O’Reilly’s Radar site, and the VLDB 2015 paper.

The key concepts in the Beam programming model are:

  • PCollection: represents a collection of data, which could be bounded or unbounded in size.
  • PTransform: represents a computation that transforms input PCollections into output PCollections.
  • Pipeline: manages a directed acyclic graph of PTransforms and PCollections that is ready for execution.
  • PipelineRunner: specifies where and how the pipeline should execute.

SDKs

Beam supports multiple language specific SDKs for writing pipelines against the Beam Model.

Currently, this repository contains SDKs for Java, Python and Go.

Have ideas for new SDKs or DSLs? See the JIRA.

Runners

Beam supports executing programs on multiple distributed processing backends through PipelineRunners. Currently, the following PipelineRunners are available:

  • The DirectRunner runs the pipeline on your local machine.
  • The ApexRunner runs the pipeline on an Apache Hadoop YARN cluster (or in embedded mode).
  • The DataflowRunner submits the pipeline to the Google Cloud Dataflow.
  • The FlinkRunner runs the pipeline on an Apache Flink cluster. The code has been donated from dataArtisans/flink-dataflow and is now part of Beam.
  • The SparkRunner runs the pipeline on an Apache Spark cluster. The code has been donated from cloudera/spark-dataflow and is now part of Beam.
  • The JetRunner runs the pipeline on a Hazelcast Jet cluster. The code has been donated from hazelcast/hazelcast-jet and is now part of Beam.

Have ideas for new Runners? See the JIRA.

Getting Started

Please refer to the Quickstart[Java, Python, Go] available on our website.

If you'd like to build and install the whole project from the source distribution, you may need some additional tools installed in your system. In a Debian-based distribution:

sudo apt-get install \
    openjdk-8-jdk \
    python-setuptools \
    python-pip \
    virtualenv

Then please use the standard ./gradlew build command.

Contact Us

To get involved in Apache Beam:

We also have a contributor's guide.

More Information