You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@pulsar.apache.org by Devin Bost <de...@gmail.com> on 2023/03/13 16:57:52 UTC

Re: Data quality problem

> Sorry. I do not fully understand here. Is it also related to the "data
> quality" problem
> that we discussed? For the consumer side, we can use the AUTO_CONSUME
schema
> to receive GenericObject (For JSON schema, you can deal with JsonObject
> directly).
> For the producer side, I think yes. We can either send an Object or
bytes[]
> (AUTO_PRODUCE).

I think there are two problems:

1. Pulsar provides no mechanism for real-time schema verification of
message content. There's no way in Pulsar to verify that the type
registered at compile-time matches the content of the message that is sent
at runtime. Companies that want to guarantee schema conformance on a
per-message basis are left to implement such a mechanism on their own.
2. The user experience around maintaining types/schemas between apps in
Pulsar is not good, but for the purpose of this thread, let's focus on that
first problem above.

Devin G. Bost


On Sun, Nov 20, 2022 at 8:02 PM PengHui Li <pe...@apache.org> wrote:

> Hi, Devin
>
> Thanks for raising the great discussion. It looks like the salient point is
> that Pulsar
> doesn't support native JSON schema. Instead, the schema is defined in the
> Avro
> standard but serialized to JSON format.JSON Schema combines aspects of
> type-based and rule-based. As this article[1] said "JSON Schema combines
> aspects of both a grammar-based language and a rule-based one". But the
> Avro
> schema definition only has the aspect of grammar-based.
>
> [1]
> https://yokota.blog/2021/03/29/understanding-json-schema-compatibility/
>
> > One of the issues with Pulsar's current implementation of schemas for
> JSON
> is the requirement to always have a POCO or some kind of type builder to
> construct the schema. This requirement can be cumbersome for users who only
> care about a few fields on the object.
>
> Sorry. I do not fully understand here. Is it also related to the "data
> quality" problem
> that we discussed? For the consumer side, we can use the AUTO_CONSUME
> schema
> to receive GenericObject (For JSON schema, you can deal with JsonObject
> directly).
> For the producer side, I think yes. We can either send an Object or bytes[]
> (AUTO_PRODUCE).
>
> > Plus, the use case is a little different compared to a DLQ or Retry
> topic because we'd like a way to handle content failures separately from
> other kinds of failures.
>
> Yes, I agree. It's not a field of DLQ.
>
> Thanks,
> Penghui
>
> On Thu, Nov 17, 2022 at 7:37 AM Devin Bost <de...@gmail.com> wrote:
>
> > I appreciate all the thoughts and questions so far.
> >
> > One of the issues with Pulsar's current implementation of schemas for
> JSON
> > is the requirement to always have a POCO or some kind of type builder to
> > construct the schema. This requirement can be cumbersome for users who
> only
> > care about a few fields on the object.
> > Protobuf attempts to simplify the implementation of the mapping (from
> data
> > to class) by having a language-independent mechanism for defining the
> data
> > (so the POCO can be generated in the desired language), but obviously,
> that
> > offers very few benefits for JSON. Additionally, protobuf and Avro don't
> > provide a way to express constraints on data *values*. Consider an
> example.
> > Let's say a site is sending messages like this:
> > {
> > "user": "bob",
> > "action": "click",
> > "trackingUrn": "urn:siteA:homepage:topNavigation:0.124",
> > "payload" : {
> >    . . .
> >    [ *highly nested or dynamic data* ]
> >    . . .
> >   }
> > }
> >
> > Here are some issues we might run into:
> > 1. A consumer wants to take action on messages based on a single field.
> > They only care about if the field exists and has an allowed value. They
> > don't want to spend a week trying to map each of the nested fields into a
> > POCO and then worry about maintaining the POCO when nested sub-fields are
> > updated by upstream teams with breaking changes. Consider these use
> cases:
> >    - Validate that the "action" value is oneOf: [ "click", "impression",
> > "hover"]. Route content based on the action unless it's an unexpected
> > value.
> >    - Subfields change depending on the trackingUrn values.
> > Consider the following:
> >    A) In the validation use case, the app developer shouldn't need to
> deal
> > with any fields other than "action", but they should be able to express
> or
> > verify that "action" is part of a data contract they have agreed to
> consume
> > from.
> >    B) Every app like this would need to add its own runtime validation
> > logic, and when many different apps are using their own versions of
> > validation, the implementations are brittle and become hard to maintain.
> > The solution to the brittleness is to adopt a standard that solves the
> > interoperability problem.
> >    C) If subfields are dynamic, well, there's not a good way to express
> > that in Avro. Maybe the developer could use maps, but I think that
> defeats
> > the purpose.
> > 2. We should be able to compose schemas from shared "schema components"
> for
> > improved reusability. (Consider it like object-oriented schema design.)
> > JSON Schema makes this possible (see detailed write-up here
> > <
> >
> https://json-schema.org/blog/posts/bundling-json-schema-compound-documents
> > >)
> > but Avro does not, so Avro schemas end up with duplication everywhere,
> and
> > this duplication is burdensome for developers to maintain. Consequently,
> > some developers avoid using schemas entirely, but that has its own
> > consequences.
> > 3. If a message's content is invalid, send the message to an "invalid
> > message topic".  Since the concerns above are mostly around data content
> at
> > runtime, Avro doesn't help us here, but for JSON content, JSON Schema's
> > validation spec
> > <
> >
> https://json-schema.org/draft/2020-12/json-schema-validation.html#name-overview
> > >
> > could. Plus, the use case is a little different compared to a DLQ or
> Retry
> > topic because we'd like a way to handle content failures separately from
> > other kinds of failures.
> >
> > (I'm sure I can think of more examples if I give it more thought.)
> >
> > Devin G. Bost
> >
> >
> > On Wed, Nov 16, 2022 at 6:36 AM 丛搏 <co...@gmail.com> wrote:
> >
> > > hi, Devin:
> > > the first Kafka doesn't support schema. `confluent `does.
> > > pulsar schema supports validation and versioning. Are you encountering
> > > a schema version caused by automatic registration, and the data source
> > > is not clear? I think you can turn off the producer's automatic
> > > registration schema, and control the schema changes through the
> > > management side.
> > > doc:
> > https://pulsar.apache.org/docs/2.10.x/schema-manage#schema-autoupdate
> > >
> > > Thanks,
> > > bo
> > >
> > > Elliot West <el...@streamnative.io.invalid> 于2022年11月14日周一
> > 20:14写道:
> > > >
> > > > While we can get caught up in the specifics of exactly how JSON
> Schema
> > is
> > > > supported in the Kafka ecosystem, it is ultimately possible if
> desired,
> > > and
> > > > is common, even if not part of open-source Apache Kafka.
> > > >
> > > > Devin's assertion is that JSON Schema compliant payload validation
> > > > and schema evolution are not currently supportable in the Pulsar
> > > ecosystem
> > > > and that perhaps they should be.
> > > >
> > > > Elliot.
> > > >
> > > >
> > > > On Fri, 11 Nov 2022 at 14:56, Elliot West <
> elliot.west@streamnative.io
> > >
> > > > wrote:
> > > >
> > > > > Hey Devin,
> > > > >
> > > > > *"Kafka conforms to the JSON Schema specification"*
> > > > > Only when using Confluent's Schema Registry.
> > > > >
> > > > > *"if a producer makes a change or omission, such as in a value used
> > for
> > > > > tracking, it might not surface until way down the line"*
> > > > > So let me understand this: Although the producer has a schema, it
> > does
> > > not
> > > > > use it for validation of JSON (as would implicitly occur for Avro?
> Is
> > > this
> > > > > correct?
> > > > >
> > > > > I agree that robust support for schema, certainly at the edges, is
> a
> > > > > cornerstone for a data system. I also agree that it would be better
> > to
> > > > > adopt existing standards rather than implement them in a bespoke
> > > manner.
> > > > >
> > > > > I'd be interested to hear your thoughts on concrete improvements
> that
> > > you
> > > > > believe would be necessary - for example:
> > > > >
> > > > > * Producer validation of JSON occurs using "JSON Schema"
> > > > > * Evolutions of JSON Schema conform to ...
> > > > > * Users can declare topic schema using a JSON Schema document
> > > > > * Users can query topic schema and have a JSON schema document
> > > returned to
> > > > > them
> > > > >
> > > > > Thanks,
> > > > >
> > > > > Elliot.
> > > > >
> > > > >
> > > > >
> > > > >
> > > > >
> > > > >
> > > > > On Thu, 10 Nov 2022 at 16:51, Devin Bost <de...@gmail.com>
> > wrote:
> > > > >
> > > > >> One of the areas where Kafka has an advantage over Pulsar is
> around
> > > data
> > > > >> quality. Kafka conforms to the JSON Schema specification, which
> > > enables
> > > > >> integration with any technology that conforms to the standard,
> such
> > > as for
> > > > >> data validation, discoverability, lineage, versioning, etc.
> > > > >> Pulsar's implementation is non-compliant with the standard, and
> > > producers
> > > > >> and consumers have no built-in way in Pulsar to validate that
> values
> > > in
> > > > >> their messages match expectations. As a consequence, if a producer
> > > makes a
> > > > >> change or omission, such as in a value used for tracking, it might
> > not
> > > > >> surface until way down the line, and then it can be very difficult
> > to
> > > > >> track
> > > > >> down the source of the problem, which kills the agility of teams
> > > > >> responsible for maintaining apps using Pulsar. It's also bad PR
> > > because
> > > > >> then incidents are associated with Pulsar, even though the
> business
> > > might
> > > > >> not understand that the data problem wasn't necessarily caused by
> > > Pulsar.
> > > > >>
> > > > >> What's the right way for us to address this problem?
> > > > >>
> > > > >> --
> > > > >> Devin Bost
> > > > >> Sent from mobile
> > > > >> Cell: 801-400-4602
> > > > >>
> > > > >
> > > > >
> > > > > --
> > > > >
> > > > > Elliot West
> > > > >
> > > > > Senior Platform Engineer
> > > > >
> > > > > elliot.west@streamnative.io
> > > > >
> > > > > streamnative.io
> > > > >
> > > > > <https://github.com/streamnative>
> > > > > <https://www.linkedin.com/company/streamnative>
> > > > > <https://twitter.com/streamnativeio>
> > > > >
> > > >
> > > >
> > > > --
> > > >
> > > > Elliot West
> > > >
> > > > Senior Platform Engineer
> > > >
> > > > elliot.west@streamnative.io
> > > >
> > > > streamnative.io
> > > >
> > > > <https://github.com/streamnative>
> > > > <https://www.linkedin.com/company/streamnative>
> > > > <https://twitter.com/streamnativeio>
> > >
> >
>

Re: Data quality problem

Posted by Devin Bost <de...@gmail.com>.
I appreciate your interest in this.

> What would the ideal data developer
> workflow look like?

Here are the first things coming to my mind on what would be ideal features:

Schema mapping code MUST be possible to auto-generate.

   - Maintaining the mapping between a data system (whether that's Pulsar
   or an RDBMS) is a very common source of errors/bugs and development
   frustration
   - Entity Framework provides a good experience here that I have not found
   with any other mapping framework.
   - Any time someone is hand-writing the mapping between data in a data
   system (whether it's Pulsar or an RDBMS) and their code, you can guarantee
   that days will be wasted in debugging mismatches, typos, missing fields,
   etc. For toy applications, it's not a big deal, but when you have hundreds
   of applications with tens or hundreds of fields with complex nesting, it
   becomes a very serious problem.

Schemas MUST be possible to auto-import into the desired language.

   - Types should be easy to generate for the target language from the
   source of truth
   - Avro claims to be able to do this, but I haven't found a way to do it
   easily. There's no good place to put hand-written Avro schemas when
   distributing them across teams. I've personally never gotten the Maven
   plugin to work (to generate Java classes from Avro types), and needing to
   write code to generate types from Avro schemas that must be written by hand
   is an unacceptable barrier for first-time users to Avro and/or Pulsar, plus
   the experience is not the same across all languages. (e.g. In Python, types
   cannot be generated from Avro, and a developer can waste incredible amounts
   of time trying to match their Python classes to the Avro schemas.) Plus,
   keeping them in sync becomes very difficult. There needs to be a single
   source of truth for schema information, and it needs to make schemas a
   first-class entity that drives all further development. The bottom line is
   that schemas should be first-class and easy to generate and use across
   languages.

Schema type inference SHOULD exist for languages that support it

   - Type inference is one of the major accelerators of productivity in a
   compiled language, so it should be well supported. It should never be the
   case that I have runtime problems because of failed type casts that should
   have instead failed compile-time checks.

Schema type checks/comparisons COULD easily be performed between Pulsar and
the IDE at compile-time

   - The ideal development experience would allow the developer to quickly
   identify type incompatibilities to catch mistakes. There are other ways
   this could be implemented, but this is one of them.

Schema changes MUST emit to a topic

   - There needs to be a way for teams to become alerted when a schema they
   care about has changed. This allows them to verify the functionality of
   their existing code (to make sure there's not a bug they missed) or update
   their code to make use of new fields.

Schema SHOULD be easily queryable for tracking schema lineage

   - For auditing and compliance, it should be possible for schemas to be
   tracked across application flows, which would enable performant
   visualization.
   - When using a regex consumer or a dynamic producer (that can produce to
   multiple topics based on content in a message header), we'd need to capture
   the type information once it hits the wire to make it possible to track
   schema lineage between applications

Schema translation into other technologies SHOULD be pluggable

   - For example, if someone wants to write in Pulsar and have it be
   readable by Oracle or Cockroach or XYZ database, it should be easy to add
   an adapter. I think Pulsar sources and sinks are almost there, but they
   suffer from gaps

Schema MUST support runtime content validation

   - Some companies are highly impacted by runtime data changes, especially
   when those changes are made by teams that exchange data across contracts or
   when function pipelines require a chain of applications to have
   interoperable schemas. (e.g. Changing the data --- not necessarily the type
   -- within one field impacts 12 other downstream apps across 3 flows and
   crosses 4 teams, and now they must all coordinate updates)
   - This validation should be easy to add without needing to put an entire
   custom processing layer on top of every topic, so ideally the validation
   framework should be knowledgable of the schema so adding validation doesn't
   require any duplication of type information. JSON Binpack is an example of
   how this could be supported.

Type information SHOULD be possible to define in one and only one place

   - Every type definition's source of truth should be defined in only one
   place so that when a change is made, it only needs to be updated in one
   place. Now, this is ideal, and it gets more tricky when trying to decouple
   systems; but, in general, defining type information once and only once
   drastically improves cyclomatic complexity and should be considered a best
   practice. How to actually accomplish this is typically the hard part.
   Functional programmers seem to have figured this out, such as with the use
   of morphisms of F-algebras and coalgebras (for example, see
   https://youtu.be/P9dbMclkD7A), but that's a bit of a digression.

Type information MUST be performant to validate without requiring
deserialization/serialization of the entire message payload

   - Flatbuffers or cap n' proto might be options here, but it would
   require some investigation.

Type information MUST be possible to express outside of message content to
ensure that content can be encrypted without hiding type information

   - It must be possible to encrypt message content without impairing the
   ability to know what the content is

GUI support COULD exist for generating type definitions

   - if the type definition language is simple enough to use, this isn't
   required

Devin G. Bost


On Tue, Mar 14, 2023 at 4:03 AM Elliot West
<el...@streamnative.io.invalid> wrote:

> Hi Devin,
>
> This topic remains of great interest to me. I think there is still a wide
> schema usability gap between traditional batch data systems (RDBMS for
> example) and those in the messaging/streaming space.
>
> Pulsar provides no mechanism for real-time schema verification of
> > message content.
>
>
> Are you specifically referring to validation at the broker entry point and
> not in the client?
>
> The user experience around maintaining types/schemas between apps in
> > Pulsar is not good
>
>
> What are we comparing this to though? What would the ideal data developer
> workflow look like?
>
> Thanks,
>
> Elliot.
>
>
> On Mon, 13 Mar 2023 at 16:58, Devin Bost <de...@gmail.com> wrote:
>
> > > Sorry. I do not fully understand here. Is it also related to the "data
> > > quality" problem
> > > that we discussed? For the consumer side, we can use the AUTO_CONSUME
> > schema
> > > to receive GenericObject (For JSON schema, you can deal with JsonObject
> > > directly).
> > > For the producer side, I think yes. We can either send an Object or
> > bytes[]
> > > (AUTO_PRODUCE).
> >
> > I think there are two problems:
> >
> > 1. Pulsar provides no mechanism for real-time schema verification of
> > message content. There's no way in Pulsar to verify that the type
> > registered at compile-time matches the content of the message that is
> sent
> > at runtime. Companies that want to guarantee schema conformance on a
> > per-message basis are left to implement such a mechanism on their own.
> > 2. The user experience around maintaining types/schemas between apps in
> > Pulsar is not good, but for the purpose of this thread, let's focus on
> that
> > first problem above.
> >
> > Devin G. Bost
> >
> >
> > On Sun, Nov 20, 2022 at 8:02 PM PengHui Li <pe...@apache.org> wrote:
> >
> > > Hi, Devin
> > >
> > > Thanks for raising the great discussion. It looks like the salient
> point
> > is
> > > that Pulsar
> > > doesn't support native JSON schema. Instead, the schema is defined in
> the
> > > Avro
> > > standard but serialized to JSON format.JSON Schema combines aspects of
> > > type-based and rule-based. As this article[1] said "JSON Schema
> combines
> > > aspects of both a grammar-based language and a rule-based one". But the
> > > Avro
> > > schema definition only has the aspect of grammar-based.
> > >
> > > [1]
> > >
> https://yokota.blog/2021/03/29/understanding-json-schema-compatibility/
> > >
> > > > One of the issues with Pulsar's current implementation of schemas for
> > > JSON
> > > is the requirement to always have a POCO or some kind of type builder
> to
> > > construct the schema. This requirement can be cumbersome for users who
> > only
> > > care about a few fields on the object.
> > >
> > > Sorry. I do not fully understand here. Is it also related to the "data
> > > quality" problem
> > > that we discussed? For the consumer side, we can use the AUTO_CONSUME
> > > schema
> > > to receive GenericObject (For JSON schema, you can deal with JsonObject
> > > directly).
> > > For the producer side, I think yes. We can either send an Object or
> > bytes[]
> > > (AUTO_PRODUCE).
> > >
> > > > Plus, the use case is a little different compared to a DLQ or Retry
> > > topic because we'd like a way to handle content failures separately
> from
> > > other kinds of failures.
> > >
> > > Yes, I agree. It's not a field of DLQ.
> > >
> > > Thanks,
> > > Penghui
> > >
> > > On Thu, Nov 17, 2022 at 7:37 AM Devin Bost <de...@gmail.com>
> wrote:
> > >
> > > > I appreciate all the thoughts and questions so far.
> > > >
> > > > One of the issues with Pulsar's current implementation of schemas for
> > > JSON
> > > > is the requirement to always have a POCO or some kind of type builder
> > to
> > > > construct the schema. This requirement can be cumbersome for users
> who
> > > only
> > > > care about a few fields on the object.
> > > > Protobuf attempts to simplify the implementation of the mapping (from
> > > data
> > > > to class) by having a language-independent mechanism for defining the
> > > data
> > > > (so the POCO can be generated in the desired language), but
> obviously,
> > > that
> > > > offers very few benefits for JSON. Additionally, protobuf and Avro
> > don't
> > > > provide a way to express constraints on data *values*. Consider an
> > > example.
> > > > Let's say a site is sending messages like this:
> > > > {
> > > > "user": "bob",
> > > > "action": "click",
> > > > "trackingUrn": "urn:siteA:homepage:topNavigation:0.124",
> > > > "payload" : {
> > > >    . . .
> > > >    [ *highly nested or dynamic data* ]
> > > >    . . .
> > > >   }
> > > > }
> > > >
> > > > Here are some issues we might run into:
> > > > 1. A consumer wants to take action on messages based on a single
> field.
> > > > They only care about if the field exists and has an allowed value.
> They
> > > > don't want to spend a week trying to map each of the nested fields
> > into a
> > > > POCO and then worry about maintaining the POCO when nested sub-fields
> > are
> > > > updated by upstream teams with breaking changes. Consider these use
> > > cases:
> > > >    - Validate that the "action" value is oneOf: [ "click",
> > "impression",
> > > > "hover"]. Route content based on the action unless it's an unexpected
> > > > value.
> > > >    - Subfields change depending on the trackingUrn values.
> > > > Consider the following:
> > > >    A) In the validation use case, the app developer shouldn't need to
> > > deal
> > > > with any fields other than "action", but they should be able to
> express
> > > or
> > > > verify that "action" is part of a data contract they have agreed to
> > > consume
> > > > from.
> > > >    B) Every app like this would need to add its own runtime
> validation
> > > > logic, and when many different apps are using their own versions of
> > > > validation, the implementations are brittle and become hard to
> > maintain.
> > > > The solution to the brittleness is to adopt a standard that solves
> the
> > > > interoperability problem.
> > > >    C) If subfields are dynamic, well, there's not a good way to
> express
> > > > that in Avro. Maybe the developer could use maps, but I think that
> > > defeats
> > > > the purpose.
> > > > 2. We should be able to compose schemas from shared "schema
> components"
> > > for
> > > > improved reusability. (Consider it like object-oriented schema
> design.)
> > > > JSON Schema makes this possible (see detailed write-up here
> > > > <
> > > >
> > >
> >
> https://json-schema.org/blog/posts/bundling-json-schema-compound-documents
> > > > >)
> > > > but Avro does not, so Avro schemas end up with duplication
> everywhere,
> > > and
> > > > this duplication is burdensome for developers to maintain.
> > Consequently,
> > > > some developers avoid using schemas entirely, but that has its own
> > > > consequences.
> > > > 3. If a message's content is invalid, send the message to an "invalid
> > > > message topic".  Since the concerns above are mostly around data
> > content
> > > at
> > > > runtime, Avro doesn't help us here, but for JSON content, JSON
> Schema's
> > > > validation spec
> > > > <
> > > >
> > >
> >
> https://json-schema.org/draft/2020-12/json-schema-validation.html#name-overview
> > > > >
> > > > could. Plus, the use case is a little different compared to a DLQ or
> > > Retry
> > > > topic because we'd like a way to handle content failures separately
> > from
> > > > other kinds of failures.
> > > >
> > > > (I'm sure I can think of more examples if I give it more thought.)
> > > >
> > > > Devin G. Bost
> > > >
> > > >
> > > > On Wed, Nov 16, 2022 at 6:36 AM 丛搏 <co...@gmail.com> wrote:
> > > >
> > > > > hi, Devin:
> > > > > the first Kafka doesn't support schema. `confluent `does.
> > > > > pulsar schema supports validation and versioning. Are you
> > encountering
> > > > > a schema version caused by automatic registration, and the data
> > source
> > > > > is not clear? I think you can turn off the producer's automatic
> > > > > registration schema, and control the schema changes through the
> > > > > management side.
> > > > > doc:
> > > >
> https://pulsar.apache.org/docs/2.10.x/schema-manage#schema-autoupdate
> > > > >
> > > > > Thanks,
> > > > > bo
> > > > >
> > > > > Elliot West <el...@streamnative.io.invalid> 于2022年11月14日周一
> > > > 20:14写道:
> > > > > >
> > > > > > While we can get caught up in the specifics of exactly how JSON
> > > Schema
> > > > is
> > > > > > supported in the Kafka ecosystem, it is ultimately possible if
> > > desired,
> > > > > and
> > > > > > is common, even if not part of open-source Apache Kafka.
> > > > > >
> > > > > > Devin's assertion is that JSON Schema compliant payload
> validation
> > > > > > and schema evolution are not currently supportable in the Pulsar
> > > > > ecosystem
> > > > > > and that perhaps they should be.
> > > > > >
> > > > > > Elliot.
> > > > > >
> > > > > >
> > > > > > On Fri, 11 Nov 2022 at 14:56, Elliot West <
> > > elliot.west@streamnative.io
> > > > >
> > > > > > wrote:
> > > > > >
> > > > > > > Hey Devin,
> > > > > > >
> > > > > > > *"Kafka conforms to the JSON Schema specification"*
> > > > > > > Only when using Confluent's Schema Registry.
> > > > > > >
> > > > > > > *"if a producer makes a change or omission, such as in a value
> > used
> > > > for
> > > > > > > tracking, it might not surface until way down the line"*
> > > > > > > So let me understand this: Although the producer has a schema,
> it
> > > > does
> > > > > not
> > > > > > > use it for validation of JSON (as would implicitly occur for
> > Avro?
> > > Is
> > > > > this
> > > > > > > correct?
> > > > > > >
> > > > > > > I agree that robust support for schema, certainly at the edges,
> > is
> > > a
> > > > > > > cornerstone for a data system. I also agree that it would be
> > better
> > > > to
> > > > > > > adopt existing standards rather than implement them in a
> bespoke
> > > > > manner.
> > > > > > >
> > > > > > > I'd be interested to hear your thoughts on concrete
> improvements
> > > that
> > > > > you
> > > > > > > believe would be necessary - for example:
> > > > > > >
> > > > > > > * Producer validation of JSON occurs using "JSON Schema"
> > > > > > > * Evolutions of JSON Schema conform to ...
> > > > > > > * Users can declare topic schema using a JSON Schema document
> > > > > > > * Users can query topic schema and have a JSON schema document
> > > > > returned to
> > > > > > > them
> > > > > > >
> > > > > > > Thanks,
> > > > > > >
> > > > > > > Elliot.
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > >
> > > > > > > On Thu, 10 Nov 2022 at 16:51, Devin Bost <devin.bost@gmail.com
> >
> > > > wrote:
> > > > > > >
> > > > > > >> One of the areas where Kafka has an advantage over Pulsar is
> > > around
> > > > > data
> > > > > > >> quality. Kafka conforms to the JSON Schema specification,
> which
> > > > > enables
> > > > > > >> integration with any technology that conforms to the standard,
> > > such
> > > > > as for
> > > > > > >> data validation, discoverability, lineage, versioning, etc.
> > > > > > >> Pulsar's implementation is non-compliant with the standard,
> and
> > > > > producers
> > > > > > >> and consumers have no built-in way in Pulsar to validate that
> > > values
> > > > > in
> > > > > > >> their messages match expectations. As a consequence, if a
> > producer
> > > > > makes a
> > > > > > >> change or omission, such as in a value used for tracking, it
> > might
> > > > not
> > > > > > >> surface until way down the line, and then it can be very
> > difficult
> > > > to
> > > > > > >> track
> > > > > > >> down the source of the problem, which kills the agility of
> teams
> > > > > > >> responsible for maintaining apps using Pulsar. It's also bad
> PR
> > > > > because
> > > > > > >> then incidents are associated with Pulsar, even though the
> > > business
> > > > > might
> > > > > > >> not understand that the data problem wasn't necessarily caused
> > by
> > > > > Pulsar.
> > > > > > >>
> > > > > > >> What's the right way for us to address this problem?
> > > > > > >>
> > > > > > >> --
> > > > > > >> Devin Bost
> > > > > > >> Sent from mobile
> > > > > > >> Cell: 801-400-4602
> > > > > > >>
> > > > > > >
> > > > > > >
> > > > > > > --
> > > > > > >
> > > > > > > Elliot West
> > > > > > >
> > > > > > > Senior Platform Engineer
> > > > > > >
> > > > > > > elliot.west@streamnative.io
> > > > > > >
> > > > > > > streamnative.io
> > > > > > >
> > > > > > > <https://github.com/streamnative>
> > > > > > > <https://www.linkedin.com/company/streamnative>
> > > > > > > <https://twitter.com/streamnativeio>
> > > > > > >
> > > > > >
> > > > > >
> > > > > > --
> > > > > >
> > > > > > Elliot West
> > > > > >
> > > > > > Senior Platform Engineer
> > > > > >
> > > > > > elliot.west@streamnative.io
> > > > > >
> > > > > > streamnative.io
> > > > > >
> > > > > > <https://github.com/streamnative>
> > > > > > <https://www.linkedin.com/company/streamnative>
> > > > > > <https://twitter.com/streamnativeio>
> > > > >
> > > >
> > >
> >
>
>
> --
>
> Elliot West
>
> Senior Platform Engineer
>
> elliot.west@streamnative.io
>
> streamnative.io
>
> <https://github.com/streamnative>
> <https://www.linkedin.com/company/streamnative>
> <https://twitter.com/streamnativeio>
>

Re: Data quality problem

Posted by Elliot West <el...@streamnative.io.INVALID>.
Hi Devin,

This topic remains of great interest to me. I think there is still a wide
schema usability gap between traditional batch data systems (RDBMS for
example) and those in the messaging/streaming space.

Pulsar provides no mechanism for real-time schema verification of
> message content.


Are you specifically referring to validation at the broker entry point and
not in the client?

The user experience around maintaining types/schemas between apps in
> Pulsar is not good


What are we comparing this to though? What would the ideal data developer
workflow look like?

Thanks,

Elliot.


On Mon, 13 Mar 2023 at 16:58, Devin Bost <de...@gmail.com> wrote:

> > Sorry. I do not fully understand here. Is it also related to the "data
> > quality" problem
> > that we discussed? For the consumer side, we can use the AUTO_CONSUME
> schema
> > to receive GenericObject (For JSON schema, you can deal with JsonObject
> > directly).
> > For the producer side, I think yes. We can either send an Object or
> bytes[]
> > (AUTO_PRODUCE).
>
> I think there are two problems:
>
> 1. Pulsar provides no mechanism for real-time schema verification of
> message content. There's no way in Pulsar to verify that the type
> registered at compile-time matches the content of the message that is sent
> at runtime. Companies that want to guarantee schema conformance on a
> per-message basis are left to implement such a mechanism on their own.
> 2. The user experience around maintaining types/schemas between apps in
> Pulsar is not good, but for the purpose of this thread, let's focus on that
> first problem above.
>
> Devin G. Bost
>
>
> On Sun, Nov 20, 2022 at 8:02 PM PengHui Li <pe...@apache.org> wrote:
>
> > Hi, Devin
> >
> > Thanks for raising the great discussion. It looks like the salient point
> is
> > that Pulsar
> > doesn't support native JSON schema. Instead, the schema is defined in the
> > Avro
> > standard but serialized to JSON format.JSON Schema combines aspects of
> > type-based and rule-based. As this article[1] said "JSON Schema combines
> > aspects of both a grammar-based language and a rule-based one". But the
> > Avro
> > schema definition only has the aspect of grammar-based.
> >
> > [1]
> > https://yokota.blog/2021/03/29/understanding-json-schema-compatibility/
> >
> > > One of the issues with Pulsar's current implementation of schemas for
> > JSON
> > is the requirement to always have a POCO or some kind of type builder to
> > construct the schema. This requirement can be cumbersome for users who
> only
> > care about a few fields on the object.
> >
> > Sorry. I do not fully understand here. Is it also related to the "data
> > quality" problem
> > that we discussed? For the consumer side, we can use the AUTO_CONSUME
> > schema
> > to receive GenericObject (For JSON schema, you can deal with JsonObject
> > directly).
> > For the producer side, I think yes. We can either send an Object or
> bytes[]
> > (AUTO_PRODUCE).
> >
> > > Plus, the use case is a little different compared to a DLQ or Retry
> > topic because we'd like a way to handle content failures separately from
> > other kinds of failures.
> >
> > Yes, I agree. It's not a field of DLQ.
> >
> > Thanks,
> > Penghui
> >
> > On Thu, Nov 17, 2022 at 7:37 AM Devin Bost <de...@gmail.com> wrote:
> >
> > > I appreciate all the thoughts and questions so far.
> > >
> > > One of the issues with Pulsar's current implementation of schemas for
> > JSON
> > > is the requirement to always have a POCO or some kind of type builder
> to
> > > construct the schema. This requirement can be cumbersome for users who
> > only
> > > care about a few fields on the object.
> > > Protobuf attempts to simplify the implementation of the mapping (from
> > data
> > > to class) by having a language-independent mechanism for defining the
> > data
> > > (so the POCO can be generated in the desired language), but obviously,
> > that
> > > offers very few benefits for JSON. Additionally, protobuf and Avro
> don't
> > > provide a way to express constraints on data *values*. Consider an
> > example.
> > > Let's say a site is sending messages like this:
> > > {
> > > "user": "bob",
> > > "action": "click",
> > > "trackingUrn": "urn:siteA:homepage:topNavigation:0.124",
> > > "payload" : {
> > >    . . .
> > >    [ *highly nested or dynamic data* ]
> > >    . . .
> > >   }
> > > }
> > >
> > > Here are some issues we might run into:
> > > 1. A consumer wants to take action on messages based on a single field.
> > > They only care about if the field exists and has an allowed value. They
> > > don't want to spend a week trying to map each of the nested fields
> into a
> > > POCO and then worry about maintaining the POCO when nested sub-fields
> are
> > > updated by upstream teams with breaking changes. Consider these use
> > cases:
> > >    - Validate that the "action" value is oneOf: [ "click",
> "impression",
> > > "hover"]. Route content based on the action unless it's an unexpected
> > > value.
> > >    - Subfields change depending on the trackingUrn values.
> > > Consider the following:
> > >    A) In the validation use case, the app developer shouldn't need to
> > deal
> > > with any fields other than "action", but they should be able to express
> > or
> > > verify that "action" is part of a data contract they have agreed to
> > consume
> > > from.
> > >    B) Every app like this would need to add its own runtime validation
> > > logic, and when many different apps are using their own versions of
> > > validation, the implementations are brittle and become hard to
> maintain.
> > > The solution to the brittleness is to adopt a standard that solves the
> > > interoperability problem.
> > >    C) If subfields are dynamic, well, there's not a good way to express
> > > that in Avro. Maybe the developer could use maps, but I think that
> > defeats
> > > the purpose.
> > > 2. We should be able to compose schemas from shared "schema components"
> > for
> > > improved reusability. (Consider it like object-oriented schema design.)
> > > JSON Schema makes this possible (see detailed write-up here
> > > <
> > >
> >
> https://json-schema.org/blog/posts/bundling-json-schema-compound-documents
> > > >)
> > > but Avro does not, so Avro schemas end up with duplication everywhere,
> > and
> > > this duplication is burdensome for developers to maintain.
> Consequently,
> > > some developers avoid using schemas entirely, but that has its own
> > > consequences.
> > > 3. If a message's content is invalid, send the message to an "invalid
> > > message topic".  Since the concerns above are mostly around data
> content
> > at
> > > runtime, Avro doesn't help us here, but for JSON content, JSON Schema's
> > > validation spec
> > > <
> > >
> >
> https://json-schema.org/draft/2020-12/json-schema-validation.html#name-overview
> > > >
> > > could. Plus, the use case is a little different compared to a DLQ or
> > Retry
> > > topic because we'd like a way to handle content failures separately
> from
> > > other kinds of failures.
> > >
> > > (I'm sure I can think of more examples if I give it more thought.)
> > >
> > > Devin G. Bost
> > >
> > >
> > > On Wed, Nov 16, 2022 at 6:36 AM 丛搏 <co...@gmail.com> wrote:
> > >
> > > > hi, Devin:
> > > > the first Kafka doesn't support schema. `confluent `does.
> > > > pulsar schema supports validation and versioning. Are you
> encountering
> > > > a schema version caused by automatic registration, and the data
> source
> > > > is not clear? I think you can turn off the producer's automatic
> > > > registration schema, and control the schema changes through the
> > > > management side.
> > > > doc:
> > > https://pulsar.apache.org/docs/2.10.x/schema-manage#schema-autoupdate
> > > >
> > > > Thanks,
> > > > bo
> > > >
> > > > Elliot West <el...@streamnative.io.invalid> 于2022年11月14日周一
> > > 20:14写道:
> > > > >
> > > > > While we can get caught up in the specifics of exactly how JSON
> > Schema
> > > is
> > > > > supported in the Kafka ecosystem, it is ultimately possible if
> > desired,
> > > > and
> > > > > is common, even if not part of open-source Apache Kafka.
> > > > >
> > > > > Devin's assertion is that JSON Schema compliant payload validation
> > > > > and schema evolution are not currently supportable in the Pulsar
> > > > ecosystem
> > > > > and that perhaps they should be.
> > > > >
> > > > > Elliot.
> > > > >
> > > > >
> > > > > On Fri, 11 Nov 2022 at 14:56, Elliot West <
> > elliot.west@streamnative.io
> > > >
> > > > > wrote:
> > > > >
> > > > > > Hey Devin,
> > > > > >
> > > > > > *"Kafka conforms to the JSON Schema specification"*
> > > > > > Only when using Confluent's Schema Registry.
> > > > > >
> > > > > > *"if a producer makes a change or omission, such as in a value
> used
> > > for
> > > > > > tracking, it might not surface until way down the line"*
> > > > > > So let me understand this: Although the producer has a schema, it
> > > does
> > > > not
> > > > > > use it for validation of JSON (as would implicitly occur for
> Avro?
> > Is
> > > > this
> > > > > > correct?
> > > > > >
> > > > > > I agree that robust support for schema, certainly at the edges,
> is
> > a
> > > > > > cornerstone for a data system. I also agree that it would be
> better
> > > to
> > > > > > adopt existing standards rather than implement them in a bespoke
> > > > manner.
> > > > > >
> > > > > > I'd be interested to hear your thoughts on concrete improvements
> > that
> > > > you
> > > > > > believe would be necessary - for example:
> > > > > >
> > > > > > * Producer validation of JSON occurs using "JSON Schema"
> > > > > > * Evolutions of JSON Schema conform to ...
> > > > > > * Users can declare topic schema using a JSON Schema document
> > > > > > * Users can query topic schema and have a JSON schema document
> > > > returned to
> > > > > > them
> > > > > >
> > > > > > Thanks,
> > > > > >
> > > > > > Elliot.
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Thu, 10 Nov 2022 at 16:51, Devin Bost <de...@gmail.com>
> > > wrote:
> > > > > >
> > > > > >> One of the areas where Kafka has an advantage over Pulsar is
> > around
> > > > data
> > > > > >> quality. Kafka conforms to the JSON Schema specification, which
> > > > enables
> > > > > >> integration with any technology that conforms to the standard,
> > such
> > > > as for
> > > > > >> data validation, discoverability, lineage, versioning, etc.
> > > > > >> Pulsar's implementation is non-compliant with the standard, and
> > > > producers
> > > > > >> and consumers have no built-in way in Pulsar to validate that
> > values
> > > > in
> > > > > >> their messages match expectations. As a consequence, if a
> producer
> > > > makes a
> > > > > >> change or omission, such as in a value used for tracking, it
> might
> > > not
> > > > > >> surface until way down the line, and then it can be very
> difficult
> > > to
> > > > > >> track
> > > > > >> down the source of the problem, which kills the agility of teams
> > > > > >> responsible for maintaining apps using Pulsar. It's also bad PR
> > > > because
> > > > > >> then incidents are associated with Pulsar, even though the
> > business
> > > > might
> > > > > >> not understand that the data problem wasn't necessarily caused
> by
> > > > Pulsar.
> > > > > >>
> > > > > >> What's the right way for us to address this problem?
> > > > > >>
> > > > > >> --
> > > > > >> Devin Bost
> > > > > >> Sent from mobile
> > > > > >> Cell: 801-400-4602
> > > > > >>
> > > > > >
> > > > > >
> > > > > > --
> > > > > >
> > > > > > Elliot West
> > > > > >
> > > > > > Senior Platform Engineer
> > > > > >
> > > > > > elliot.west@streamnative.io
> > > > > >
> > > > > > streamnative.io
> > > > > >
> > > > > > <https://github.com/streamnative>
> > > > > > <https://www.linkedin.com/company/streamnative>
> > > > > > <https://twitter.com/streamnativeio>
> > > > > >
> > > > >
> > > > >
> > > > > --
> > > > >
> > > > > Elliot West
> > > > >
> > > > > Senior Platform Engineer
> > > > >
> > > > > elliot.west@streamnative.io
> > > > >
> > > > > streamnative.io
> > > > >
> > > > > <https://github.com/streamnative>
> > > > > <https://www.linkedin.com/company/streamnative>
> > > > > <https://twitter.com/streamnativeio>
> > > >
> > >
> >
>


-- 

Elliot West

Senior Platform Engineer

elliot.west@streamnative.io

streamnative.io

<https://github.com/streamnative>
<https://www.linkedin.com/company/streamnative>
<https://twitter.com/streamnativeio>