-
Notifications
You must be signed in to change notification settings - Fork 4.3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[BEAM-7029] Add KafkaIO.Read as external transform #8251
Conversation
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
LGTM. Found some typos.
|
||
args = { | ||
'consumer_config': | ||
KafkaRead._encode_map(self.consumer_config), |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
KafkaRead -> ReadFromKafka
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ah, thanks. I renamed this before opening the PR and missed those.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I haven't caught up on all the cross language pipeline stuff but for this PR and overall this CL looks fine based upon the current expansion service design. Maybe you could point me to some docs/discussions about some of the ideas/concerns I have below:
I would have expected that the loss of typing information in generics is typically going to cause you to not be able to "automatically" call setters on builders in any practical way. Also, using automatic conversion from name to method will mean that the API change will break compatibility with the "proto" version requiring people to providing a remapping feature anyways.
I would use explicit protos for each transform like:
message KafkaRead {
map<string, string> consumer_config = 1;
repeated string topics = 2;
string key_deserializer = 3;
string value_deserializer = 4;
}
that are defined inside a proto package like "org.apache.beam.external.kafka.v1"
.
Using an explicit proto would allow you to change the builder method to be annotated with the URN and then you would be able to automatically decode the payload and pass it into the method since you could look at the method parameter. This would also allow the Java SDK to create a transform with a payload that is this proto instead of serialized Java objects.
Also, how do you "choose" what the output type of the PCollection produced by Kafka.Read
is?
If someone passes in the org.apache.kafka.common.serialization.LongDeserializer
like in your example, the returned PCollection may contain some Java SDK specific coder. Would it make sense to have a mapping from beam coder URNs to kafka deserializers instead?
Note, that these questions may be better for the mailing list then this review if this is still up for discussion.
Thanks for taking a look @lukecwik. The design document for this is https://s.apache.org/beam-cross-language-io. Approach (2) in the design doc was the original proposal for configuring cross-language transforms. It is very close to what you are suggesting. The PR #7875 originally had this design, but we changed it to match approach (1) after a discussion in the PR, on the mailing list, and in a meeting which @chamikaramj summarized on the mailing list. It is not necessarily a contradiction to support both, an explicit Proto for each transform, and a generic approach. However, the main objective for cross-language transforms was that there should not be the need for users to write Proto. So the generic approach was deemed more important.
Good point. I think it makes sense to define a mapping of supported coders for Kafka deserializers, since we can only support standard SDK coders. |
Run Java PreCommit |
Flaky Hadoop-related test: https://builds.apache.org/job/beam_PreCommit_Java_Commit/5269/console |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks @mxm.
@@ -172,6 +173,20 @@ def test_external_transform(self): | |||
|
|||
assert_that(res, equal_to([i for i in range(1, 10)])) | |||
|
|||
# We do not run KafkaIO but check that the transform | |||
# is expanded by the ExpansionService without an error. | |||
p = self.create_pipeline() |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can we also add a validator to validate output ?
@ihji KafkaIO will be a great addition to validates runner test suite you are adding.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
There are tests which check the correct configuration of KafkaIO. This just checks that the expansion succeeds. We definitely want a full end-to-end test. I have to figure out how to start an embedded Kafka cluster here.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sg. I think a full end-to-end test will definitely be helpful.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Considering that KafkaIO itself is only mock-tested, I'm not sure I want to bring up a Kafka cluster here. I think we have sufficient test coverage for the configuration through KafkaIOExternalTest
. We could execute here and check for an exception, e.g. "Unknown broker".
| ReadFromKafka(consumer_config={'bootstrap.servers': | ||
'localhost:94583, localhost:3531'}, | ||
topics=['topic1', 'topic2'], | ||
key_deserializer='org.apache.kafka.common.' |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Probably document this as a significant limitation. We have to specify Java functions from Python till we support UDFs.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Pushed an update with a better documentation.
'localhost:94583, localhost:3531'}, | ||
topics=['topic1', 'topic2'], | ||
key_deserializer='org.apache.kafka.common.' | ||
'serialization.BytesDeserializer', |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
How can a user make a new Java function available for Python pipeline deployment. Please document this.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It's currently not easily possible. We have to rely on the built-in ones for now. Have documented this.
from apache_beam.transforms import ptransform | ||
|
||
|
||
class ReadFromKafka(ptransform.PTransform): |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Please add a doc-comment describing the interface of Python Kafka transform and current limitations.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The doc comment is below. I expanded it to contain more information about the status quo.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks. I think we should also provide end-to-end instructions on how a user can use this transform in a pipeline, including for example, steps for starting up the expansion service (here or elsewhere) given the this is the first real world source that some of our users might try to use out of the box in the next Beam release.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Does it make sense to include the instructions for getting started with cross-language portability here? We can add an outline here of what the user has to do. For detailed instructions, there should be a page on the Beam homepage. The only page I could think of at the moment would be https://beam.apache.org/contribute/portability/. We should have a more user-friendly page for the next release.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think yes. Otherwise users will have no way of knowing how to use this in a pipeline. I think the instructions can be as simple as steps to get the expansion service running (or a pointer to a script for doing this) and providing the address to the ReadFromKafka transform (no need to mention runner setup).
I agree that when things are more stable, we should develop a Webpage for cross-language transforms with complete instructions, examples (for transform authors and users), etc. (not in this PR).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I've included a pointer to the Flink Job Server which starts the Java expansion service. Additionally, I've linked documentation pages which explain how to start the Flink Job Server. Should be sufficient for now.
raise Exception("ReadFromKafka must be a root transform") | ||
|
||
args = { | ||
'consumer_config': |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can we also support write ?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sure. I wanted to start off with Read but I can add Write here or in a seperate PR.
|
||
class ReadFromKafka(ptransform.PTransform): | ||
""" | ||
A PTransform which from Kafka topics. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Also, please mention currently supported runners (Flink) to prevent user-confusion.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Good point.
@lukecwik I summarized the current design (adapter from Max's original design doc) in an email to dev list: https://lists.apache.org/thread.html/b99ba8527422e31ec7bb7ad9dc3a6583551ea392ebdc5527b5fb4a67@%3Cdev.beam.apache.org%3E The idea is to minimize the amount of work a transform author has to do to make a given transform available to remote SDKs. With the current solution, the transform authors simply have to provide a builder that can be used to build the transform from a given config object (POJO). Framework will generate a proto from this that defines the wire format. Remote SDKs can invoke arbitrary transforms that are exposed through the expansion API (and has a builder available) by using generic ExternalTransform. Also, transform authors may define nicer wrappers (of ExternalTransform) if they choose to do so. Of course this means that a cross-SDK interaction can only be done thorough standard coders that are defined for both languages. These will be used for (1) encoding parameters used to build the rewmote transform (2) encoding data transferred between SDKs at runtime. A pipeline author has to be aware of this and has to make sure that transforms at SDK boundaries use standard coders. We hope the solution will become more generic once Beam SQL becomes available (where we should be able to do a Row to Row conversion from SDK A to SDK B for generic types). |
Run Python PreCommit |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks. LGTM other than few comments.
} | ||
} | ||
|
||
/** Exposes GenerateSequence as an external transform for cross-language usage. */ |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
s/GenerateSequence/KafkaIO.TypedWithoutMetadata
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Fixed.
from apache_beam.transforms import ptransform | ||
|
||
|
||
class ReadFromKafka(ptransform.PTransform): |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks. I think we should also provide end-to-end instructions on how a user can use this transform in a pipeline, including for example, steps for starting up the expansion service (here or elsewhere) given the this is the first real world source that some of our users might try to use out of the box in the next Beam release.
@@ -172,6 +173,20 @@ def test_external_transform(self): | |||
|
|||
assert_that(res, equal_to([i for i in range(1, 10)])) | |||
|
|||
# We do not run KafkaIO but check that the transform | |||
# is expanded by the ExpansionService without an error. | |||
p = self.create_pipeline() |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sg. I think a full end-to-end test will definitely be helpful.
Thanks @mxm and @chamikaramj for the links. I read through them and have a better understanding of the details. Using the decoded PCollection<keyType, valueType> for KafkaIO.Read will cover many user usecases but from experience with the GCP pubsub native source for Dataflow and the Python SDK users always wanted to get additional attributes from the pubsub message and only passing through the "data" meant that this didn't satisfy what many users wanted (and exposing a few additional attributes at a time wasn't great). This was easy for pubsub since they used proto as the canonical wire format. As a future follow-up it may be useful to also expose a Java KafkaIO.Read that produces org.apache.kafka.common.record.Record encoded as a byte[]. This would mean that the output type of KafkaIO.Read for the cross language transform is PCollection<byte[]> and push all the decoding logic into the Python SDK. This would mean that the user could use any "deserializer" they want but this puts a greater burden on the language that wants to consume the records since they will need to be able to decode such messages. Many times this additional logic in the downstream SDK isn't difficult to implement since you can rely on a language specific source library to do the parsing. Alternatively, if there aren't many language specific libraries for the source format, it may be wise to produce an intermediate format such as proto/json/... (which can generate language specific bindings or are very well supported in almost all languages) which sends all the data across. |
Makes sense in general. If a IO connector offers a transform that produces records as bytes, it makes sense to support pushing deserializers to remote SDKs. Looks like there's no such a read transform for Java |
@lukecwik That's correct. During implementation I discovered that data is always wrapped into
I think we will have to support Read in portable Runners for now until SDF is finalized and an adapter for UnboundedSource is available. For the Flink Runner this is already the case and other Runners can add it easily. It's not so bad because, after all, it is transparent to the user. |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks for your comments @chamikaramj.
} | ||
} | ||
|
||
/** Exposes GenerateSequence as an external transform for cross-language usage. */ |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Fixed.
from apache_beam.transforms import ptransform | ||
|
||
|
||
class ReadFromKafka(ptransform.PTransform): |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Does it make sense to include the instructions for getting started with cross-language portability here? We can add an outline here of what the user has to do. For detailed instructions, there should be a page on the Beam homepage. The only page I could think of at the moment would be https://beam.apache.org/contribute/portability/. We should have a more user-friendly page for the next release.
@@ -172,6 +173,20 @@ def test_external_transform(self): | |||
|
|||
assert_that(res, equal_to([i for i in range(1, 10)])) | |||
|
|||
# We do not run KafkaIO but check that the transform | |||
# is expanded by the ExpansionService without an error. | |||
p = self.create_pipeline() |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Considering that KafkaIO itself is only mock-tested, I'm not sure I want to bring up a Kafka cluster here. I think we have sufficient test coverage for the configuration through KafkaIOExternalTest
. We could execute here and check for an exception, e.g. "Unknown broker".
@mxm Supporting the whole KafkaRecord could be a TODO for the future. For now you could just produce KV<byte[], byte[]> with the intent that the receiving end performs any Kafka key/value -> language specific type decoding (instead of passing in the key / value deserializers to the Java SDK). This will produce a version that has a lot of value for many people since all "data" types can be supported in the Python SDK by exposing a UDF that users can implement to perform the decoding in the Python SDK. You could produce some defaults for things like bytes/utf8/dates/... Also, to support the whole KafkaRecord in the future, Kafka must already have a well defined "encoding" format for compatibility across all its client SDKs so it would be great if we could use that and then use the parsing in the Kafka client libraries to do that work. Our coder spec would be beam:coder:kafka_record:v1 and just point to a certain version of the KafkaRecord spec. |
@lukecwik I think your experience with Pubsub on Dataflow is a good reminder of the requirements users have when reading from a message queue. For simple use cases, only having the data might be fine but ultimately users will ask for the metadata (partition id, partition offset, timestamp). Defining a standard coder for this seems inevitable. For now, going with the current approach here to just ship the data seems to be fine. The I'm wondering whether we should keep the mapping from Kafka Deserializer to standard coders. Most Kafka users are familiar with Java and are used to Kafka Deserializers. The obvious drawback of this approach is that we need to maintain this mapping and we can't support all Deserializers. We could make @lukecwik @chamikaramj WDYT? |
I think it makes sense to keep a mapping from deserializers to standard coders in Python SDK. This way users can configure next step of the python pipeline to consume data using the corresponding coder. The issue (as you mentioned) is keeping such a list up to date with what happens in the Java SDK. But probably changes to Kafka deserializers are rare enough to justify this. |
I agree with @chamikaramj about making the decoding happen within the Python SDK for the same reason. |
This adds KafkaIO.Read as an external transform and includes a Python wrapper (ReadFromkafka) for convenience. The transform only returns the key/value data for a Kafka topic. It does not include the meta data such as partition id, offset, or timestamp. By default, the data is returned as `KV<byte[], byte[]>`. Users can supply a Kafka Deserializer in ReadFromkafka such as LongDeserializer which will infer a different coder. Only a limited amount of Deserializers are supported. Alternatively, users can implement their own decoding in the target SDK.
I've made |
LGTM. Thanks. |
Thanks for the reviews! |
Unrelated |
@mxm I've been trying to use the ReadFromKafka transform (kafka.py) via Flink with the portable runner, but it seems the value coder resolves to ByteArrayCoder instead of KafkaRecordCoder. Am I missing a registration step? My transform is similar to the one in the unit test (flink_runner_test.py). Here are some logs with some extra INFO I added:
|
Thanks for trying out the Kafka consumer. There are limitations for the consumer to work correctly. The biggest issue is the structure of KafkaIO itself, which uses a combination of the source interface and DoFns to generate the desired output. The problem is that the source interface is natively translated by Flink to support unbounded sources in portability, while the DoFn runs in a Java environment. To transfer data between the two a coder needs to be involved. It happens to be that the initial read does not immediately drop the KafakRecord structure which does not work together well with our current assumption of only supporting "standard coders" present in all SDKs. There are several possible solutions:
For a workaround which uses (3), please see this patch which is not a proper fix but adds KafkaRecordCoder to the SDK such that it can be used encode/decode records: mxm@b31cf99 I'm planning to fix the issue in the coming weeks. |
Created a JIRA issue: https://jira.apache.org/jira/browse/BEAM-7870 |
@mxm I tried the patch and I can see that the right coder is applied to the transform. Thank you. My next step is to have the transform output (KafkaRecords) to be passed to my Python pipeline. How can I handover KafkaRecord elements from Java to Python? Will it work by calling an external transform in the Python pipeline or is there a simpler way to translate the ReadFromKafka output into a Python implementation of KafkaRecord (e.g. confluent_kafka). For my pipeline test, I added a Map function 'passthrough' (it just logs the message and yields the element) in my Python pipeline after reading from Kafka:
As of now, this is the error I'm getting while the portable runner is reading from Kafka topics
|
This error occurs when the output coder can not be inferred correctly from the Python function. The following should work: messages | 'Passthrough' >> beam.Map(passthrough).with_output_types(KV[bytes, bytes]) Apart from this, no additional work should be necessary to make this work. Let me know if that fixes the problem you are seeing. Thanks for the feedback. |
@mxm I tried the hinted type but I'm still getting the same error. 'passthrough' is a local Python function:
And the error:
Does portability allow having both external and local transforms in a pipeline in different languages? From what I read, the pipeline is submitted to the runner for execution, how does the runner handle cross language pipelines? In this case, the execution of the pipeline occurs in the Flink server (Java), and it's not expecting a Python urn. |
@mxm By looking at the payload, I think I introduced an issue when hardcoding SDK docker image names in the fn executor docker factory (my organization uses the very unusual id@domain as Unix user name and docker rejects such image names). |
Yes it does. You can mix Python, Java code, and Go code in the same pipeline. Each have their own environment via docker containers. That's also why we are limited to standard coders to transfer data between the different environments. It is a bit tricky though for sources where we actually mix three different environments a) Java via docker b) Python via docker c) native Java via the Flink runner to run the old style unbounded sources, i.e. Kafka IO. The latter code path will be removed once we have an adapter to run old style sources via SplittableDoFn. If that does not make sense to you, that's alright :) Not sure what the issue is here. Earlier I ran a semantically identical pipeline with a ReadFromKafka and a pass-through Python transform, followed by a WriteToKafka. Will take a closer look when I get a chance (currently traveling). |
@mxm The issue was related to invalid docker image names. My first issue (invalid docker image name) was solved by creating a plain Unix account and building all the images there. A small issue I noticed after was when deploying the portable runner in Linux for a pipeline that runs multiple languages (Java + Python). The flink server does not have the USER environment variable defined, so it can't find the right payload image to use on the second transform and probably onwards. I hardcoded the USER environment variable in the flink server Dockerfile and it now finds the right image for the right payload. The patch works and I'm able to consume Kafka payload using the portable runner. Another thing I noticed is that although the deserializer I specified in the ReadFromKafka transform is for ByteArray (both for key and value), the Python pipeline is consuming it as a string without further conversion. Would it make sense to have the KafkaRecord metadata as an option in the portable pipeline? (to be able to access partitions, offsets, timestamps, and other KafkaRecord properties). I'll continue running some performance tests to see how Flink scales up with a few million records in the portability framework. |
Great to hear that you resolved the issue. The user variable could probably be inferred from the user's home directory.
The string consumption will only work for Python 2 though where bytes and strings are treated equally.
That would make perfect sense. However, we wanted to avoid having to include KafkaRecord as a type across SDKs. We figured it might be sufficient to have only the raw data available. As you saw, it is not yet possible to completely use the KafkaIO consumer without the KafkaRecord cross-language coder. Hopefully, we will have fixed that soon and introduce a structure for preserving KafkaRecord meta data across SDKs. Curious to learn about your performance tests. |
@mxm It seems performance depends mostly on the sink. I've been able to get 3000 msg/sec with a file sink (which doesn't complete the final write when I cancel the job via Flink), and 2300 msg/sec with a recently patched version of the mongodb sink. The pipeline had a map transform to get the value from every KV element. This was done on a quad core Intel(R) Xeon(R) CPU E5-2695 v4 @ 2.10GHz virtual machine with 8GB memory using a standalone Flink docker image as runner endpoint. |
This adds KafkaIO.Read as an external transform and includes a Python wrapper (ReadFromkafka) for convenience.
CC @tweise @robertwb @ihji @lukecwik
Post-Commit Tests Status (on master branch)
Pre-Commit Tests Status (on master branch)
See .test-infra/jenkins/README for trigger phrase, status and link of all Jenkins jobs.