Confluent Cloud スキーマ管理のドキュメントをお探しですか? これらのページでは、Schema Registry の全体的な概念、スキーマフォーマット、ハイブリッドのユースケース、チュートリアル といったポイントに触れていますが、最も重点的に取り上げているのは Confluent Platform です。Confluent Cloud のドキュメントについては、Confluent Cloud でのスキーマの管理」をご覧ください。
Protobuf スキーマのシリアライザーと逆シリアライザー¶
このドキュメントでは、Apache Kafka® の Java クライアントやコンソールツールで Protocol Buffers(Protobuf) を使用する方法について説明します。
Protobuf シリアライザー¶
Protobuf 型のメッセージを Kafka に送信するには、KafkaProducer
に KafkaProtobufSerializer
をプラグインします。
Protobuf によって生成されたクラスのインスタンスをシリアライザーに渡すと、シリアライザーは、Protobuf スキーマとすべての参照先スキーマを登録することができます。デフォルトでは、参照先の各スキーマが、その参照と同じ名前のサブジェクトで登録されます。たとえば、メインスキーマが "google/protobuf/timestamp.proto
" を参照する場合、その timestamp スキーマが "google/protobuf/timestamp.proto
" という名前のサブジェクトで登録されます。スキーマフォーマットの「構成の詳細」の説明にあるように、この動作は、reference.subject.name.strategy
構成を使用して、ReferenceSubjectNameStrategy
のカスタム実装をシリアライザーに渡すことでカスタマイズできます。
そのインターフェイスは、次のようになっています。
public interface ReferenceSubjectNameStrategy extends Configurable
{
/**
* For a given reference name, topic, and message, returns the subject name under which the
* referenced schema should be registered in the schema registry.
*
* @param refName The name of the reference.
* @param topic The Kafka topic name to which the message is being published.
* @param isKey True when encoding a message key, false for a message value.
* @param schema The referenced schema.
* @return The subject name under which the referenced schema should be registered.
*/
String subjectName(String refName, String topic, boolean isKey, ParsedSchema schema);
}
たとえば、次のようなスキーマがあるとします。
syntax = "proto3";
package com.acme;
import "other.proto";
message MyRecord {
string f1 = 1;
OtherRecord f2 = 2;
}
上のスキーマは other.proto
(下記)を参照しています。
syntax = "proto3";
package com.acme;
message OtherRecord {
int32 other_id = 1;
}
以下のコードは、Protobuf コンパイラーによって生成される MyRecord
クラスのインスタンスを作成します。Kafka プロデューサーは、Protobuf シリアライザーを使用して MyRecord
インスタンスをシリアル化するように設定されています。
ちなみに
以下の例では、Kafka ブートストラップサーバーと Schema Registry にデフォルトのアドレスとポート(それぞれ localhost:9092
および localhost:8081
)が使用されています。
Properties props = new Properties();
props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG,
"org.apache.kafka.common.serialization.StringSerializer");
props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG,
"io.confluent.kafka.serializers.protobuf.KafkaProtobufSerializer");
props.put("schema.registry.url", "http://127.0.0.1:8081");
Producer<String, MyRecord> producer = new KafkaProducer<String, MyRecord>(props);
String topic = "testproto";
String key = "testkey";
OtherRecord otherRecord = OtherRecord.newBuilder()
.setOtherId(123).build();
MyRecord myrecord = MyRecord.newBuilder()
.setF1("value1").setF2(otherRecord).build();
ProducerRecord<String, MyRecord> record
= new ProducerRecord<String, MyRecord>(topic, key, myrecord);
producer.send(record).get();
producer.close();
上のコードは、Kafka に対して Protobuf インスタンスをシリアル化する際、MyRecord
用と OtherRecord
用の 2 つのスキーマを、2 つの異なるサブジェクトで Schema Registry に自動的に登録します(スキーマを自動的に登録するというデフォルトの動作は、auto.register.schemas=false
プロパティをシリアライザーに渡すことで無効にできます)。
では、登録されたスキーマを、REST エンドポイントを使用して調べてみましょう。まず、どのサブジェクトが使用されているかは、次のコマンドを使用して確認できます。
curl http://localhost:8081/subjects
次のような結果が出力されます。
["testproto-value", "other.proto"]
トップレベルスキーマのサブジェクトは SubjectNameStrategy
によって決まり、デフォルトでは、トピックにサフィックスとして -key
または -value
を付けたものになります。参照先スキーマのサブジェクトは ReferenceSubjectNameStrategy
によって決まり、デフォルトでは、import ステートメントで使用されている名前になります。どちらの命名方法もカスタマイズすることができます。
以下のスキーマにある新しい schemaType
フィールドに注目してください。これは Confluent Platform 5.5 向けに追加されたものです。また、トップレベルスキーマには、other.proto
を参照する新しい references
フィールドが存在します。
testproto-value
スキーマを表示するには、次のコマンドを入力します。
curl http://localhost:8081/subjects/testproto-value/versions/1
次のような結果が出力されます。
{
"subject": "testproto-value",
"version": 1,
"id": 2,
"schemaType": "PROTOBUF",
"references": [
{
"name": "other.proto",
"subject": "other.proto",
"version": 1
}
],
"schema": "syntax = \"proto3\";\npackage com.acme;\n\nimport \"other.proto\";\n\nmessage MyRecord {\n string f1 = 1;\n .com.acme.OtherRecord f2 = 2;\n}\n"
}
testproto-value
のスキーマ ID が 2
で、other.proto
のスキーマ ID が 1
になっていることに注意してください。
other.proto
スキーマを表示するには、次のコマンドを入力します。
curl http://localhost:8081/subjects/other.proto/versions/1
次のような結果が出力されます。
{
"subject": "other.proto",
"version": 1,
"id": 1,
"schemaType": "PROTOBUF",
"schema": "syntax = \"proto3\";\npackage com.acme;\n\nmessage OtherRecord {\n int32 other_id = 1;\n}\n"
}
今度は、参照先スキーマである other.proto
を削除してみます。
curl -X DELETE http://localhost:8081/subjects/other.proto/versions/1
次のエラーが発生します。Schema Registry では、不確定な参照を作成できないようになっているためです。トップレベルのスキーマよりも先に参照先スキーマを削除することは、不確定な参照に該当します。
{"error_code":42206,"message":"One or more references exist to the schema {magic=1,keytype=SCHEMA,subject=other.proto,version=1}."}
ご存じのように、今削除しようとしたスキーマは、ID 2
のスキーマによって参照されています。その情報は、次のコマンドを使用して直接、Schema Registry から取得することもできます。
curl http://localhost:8081/subjects/other.proto/versions/1/referencedby
上のコマンドからは、other.proto
スキーマを参照する testproto-value
スキーマの ID(ID 2
)が返されます。
[2]
1 つのスキーマが複数のサブジェクトに存在する場合もあるので、それぞれのサブジェクトからスキーマを削除する必要があります。各サブジェクトからスキーマを削除した後、参照先スキーマを削除することができます具体的にどこでスキーマ ID が使用されているかは、次のようにして調べることができます。
curl http://localhost:8081/schemas/ids/2/versions
上のコマンドからは、次の結果が返されます。このケースでは、other.proto
を参照するスキーマが 1 つしかないことがわかります。
[{"subject":"testproto-value","version":1}]
other.proto
を参照しているトップレベルのスキーマ(testproto-value
)を削除します。
curl -X DELETE http://localhost:8081/subjects/testproto-value/versions/1
単純に 1
が出力されると思います。
これで参照先スキーマを削除できるようになりました。コマンドは正常に実行されます。
curl -X DELETE http://localhost:8081/subjects/other.proto/versions/1
単純に 1
が出力されると思います。
Protobuf デシリアライザー¶
Protobuf 型のメッセージを Kafka から受信するには、KafkaConsumer
に KafkaProtobufDeserializer
をプラグインします。
ちなみに
以下の例では、Kafka ブートストラップサーバーと Schema Registry にデフォルトのアドレスとポート(それぞれ localhost:9092
および localhost:8081
)が使用されています。
Properties props = new Properties();
props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
props.put(ConsumerConfig.GROUP_ID_CONFIG, "group1");
props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer");
props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "io.confluent.kafka.serializers.protobuf.KafkaProtobufDeserializer");
props.put("schema.registry.url", "http://localhost:8081");
props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");
String topic = "testproto";
final Consumer<String, Message> consumer = new KafkaConsumer<String, Message>(props);
consumer.subscribe(Arrays.asList(topic));
try {
while (true) {
ConsumerRecords<String, Message> records = consumer.poll(100);
for (ConsumerRecord<String, Message> record : records) {
System.out.printf("offset = %d, key = %s, value = %s \n", record.offset(), record.key(), record.value());
}
}
} finally {
consumer.close();
}
Protobuf ペイロードを逆シリアル化するとき、KafkaProtobufDeserializer
は次のように動作します。
specific.protobuf.key.type
またはspecific.protobuf.value.type
が指定されている場合、デシリアライザーは、その指定された型を使用して逆シリアル化を実行します。RecordNameStrategy
では、Protobuf のderive.type=true
から複数の型が導出され、先ほどの設定がうまく機能しません。この場合はさらに、java_outer_classname
またはjava_multiple_files = true
を元の Protobuf ファイルで指定する必要があります。これによってデシリアライザーは Java の型をスキーマから導出し、Protobuf ペイロードを逆シリアル化することができます。- 最後に、型が指定されなかった場合や型を導出できなかった場合、デシリアライザーは、スキーマを使用して Protobuf
DynamicMessage
のインスタンスを返します。
Avro デシリアライザーが具体的な Avro レコード型のインスタンスまたは GenericRecord
を返すように、Protobuf デシリアライザーも、具体的な Protobuf メッセージ型のインスタンスまたは DynamicMessage
を返します。
Protobuf デシリアライザーが具体的な型を特定できなかった場合は、汎用的な型が返されます。
具体的な型を返す 1 つの方法は、明示的なプロパティを使用することです。Protobuf デシリアライザーの場合、KafkaProtobufDeserializerConfig.SPECIFIC_PROTOBUF_VALUE_TYPE
プロパティまたは KafkaProtobufDeserializerConfig.SPECIFIC_PROTOBUF_KEY_TYPE
プロパティを設定することができます。
Protobuf デシリアライザーで異種混合の型を含んだトピックを扱うためには、スキーマに追加情報を指定する必要があります。derive_type=true
でデシリアライザーを設定し、そのうえで java_outer_classname
または java_multiple_files=true
をスキーマの中で指定してください。
以下の表は、返される具体的な型と汎用的な型をスキーマ形式ごとにまとめたものです。
Avro | Protobuf | JSON Schema | |
---|---|---|---|
具体的な型 | 生成されたクラス(org.apache.avro.SpecificRecord を実装) | 生成されたクラス(com.google.protobuf.Message を拡張) | Java クラス(Jackson シリアル化との互換性あり) |
汎用的な型 | org.apache.avro.GenericRecord | com.google.protobuf.DynamicMessage | com.fasterxml.jackson.databind.JsonNode |
Protobuf Schema の体験¶
実際に Protobuf を使ってみましょう。Protobuf では、コマンドラインのプロデューサーとコンシューマーを使用できます。Avro と同様、Protobuf にも、バイナリシリアル化形式と JSON シリアル化形式の両方が定義されています。人間が読めるようにしたければ JSON を使用し、トピックへのデータの格納効率を優先したければバイナリ形式を使用することができます。
注釈
- 以下の例を実行するための前提条件は、 Schema Registry のチュートリアル に記載されている条件とほぼ同じです。ただし、ここでは Maven は必要ありません。また、ここでは Confluent Platform バージョン 5.5.0 以降が必要となります。
- 以降の例では、Schema Registry の URL にデフォルト値(
localhost:8081
)が使用されています。これらの例を見ると、その URL をプロデューサーとコンシューマーのコマンドライン引数の--property
フラグに引数として指定することにより、インラインでこの値を設定していることがわかります(--property schema.registry.url=<実際の Schema Registry のアドレス>
)。別の方法として、このプロパティを$CONFLUENT_HOME/etc/kafka/server.properties
で設定することもできます。そうすれば、プロデューサーとコンシューマーのコマンドで URL を指定する必要はありません。たとえば、「confluent.schema.registry.url=http://localhost:8081
」のように指定します。
以下の例では、$CONFLUENT_HOME/bin
にある kafka-protobuf-console-producer
と kafka-protobuf-console-consumer
を利用しています。
コマンドラインのプロデューサーとコンシューマーは、Confluent Platform における組み込みの Protobuf スキーマ機能の働きを理解するうえで役立ちます。
実際にプロデューサーとコンシューマーのコードにシリアライザーとデシリアライザーを組み込んだときにも、コンソールから実行したときと同じようにメッセージおよび関連するスキーマが処理されます。
プロデューサーを実行した直後にコンシューマーを実行できなかった場合でも確実にメッセージを取り込むために、コンシューマーコマンドには、先頭から読み取るためのフラグ(--from-beginning
)を指定することをお勧めします。--from-beginning
フラグを省略した場合、コンシューマーは、現在のセッション中に生成された最後のメッセージしか読み取りません。
次のコマンドを使用して Confluent Platform を起動します。
confluent local services start
ちなみに
- 単に
confluent local schema-registry
を実行してもかまいません。その場合は、依存関係としてkafka
とzookeeper
も起動されます。このデモでは、Connect や Control Center など、他のサービスを直接参照することはしていません。もっとも、さらに理解を深めるために(トピックやメッセージが Control Center にどのように表示されるかなど)、フルスタックを実行したい場合もあるでしょう。confluent local
の詳細については、「Confluent Platform のクイックスタート(ローカルインストール)」および Confluent CLI コマンドリファレンスの「confluent local」を参照してください。 confluent local
のコマンドはバックグラウンドで実行されるので、そのコマンドウィンドウを再利用することができます。プロデューサーとコンシューマーのセッションは独立している必要があります。
- 単に
登録されているスキーマタイプを確認します。
Confluent Platform 5.5.0 以降では、Schema Registry で任意のスキーマタイプがサポートされます。どのタイプのスキーマが現在 Schema Registry に登録されているかを確認する必要があります。
そこで、次のコマンドを入力します(Schema Registry の URL とポートには、デフォルトの
localhost:8081
が使用されているものとします)。curl http://localhost:8081/schemas/types
次のいずれかまたは複数の項目が応答として返されます。さらに、スキーマ形式プラグインがインストールされている場合は、それらも候補となります。
["JSON", "PROTOBUF", "AVRO"]
または、
curl --silent
フラグを使用し、コマンドをパイプで jq に連結すれば(curl --silent http://localhost:8081/schemas/types | jq
)、整った書式の出力結果を得ることができます。"JSON", "PROTOBUF", "AVRO"
プロデューサーを使用して、JSON 形式の Protobuf レコードをメッセージの値として送信します。
トピック
t1-p
がまだ存在しない場合、このプロデューサーコマンドの実行過程で新たに作成されます。kafka-protobuf-console-producer --broker-list localhost:9092 \ --property schema.registry.url=http://localhost:8081 --topic t1-p \ --property value.schema='syntax = "proto3"; message MyRecord { string f1 = 1; }'
ちなみに
現在の Protobuf 用プロデューサーには、
>
プロンプトが表示 "されません"。空の行が表示されるだけです。そこに、プロデューサーのメッセージを入力することになります。シェルに次のコマンドを入力し、Return キーを押します。
{"f1": "value1-p"}
コマンドライン Protobuf プロデューサーが(
<value.schema>
に指定されたスキーマを使用して)JSON オブジェクトを Protobuf メッセージに変換し、ベースとなるシリアライザーを使用して、Kafka トピックt1-p
にメッセージをシリアル化します。コンシューマーを使用して、トピック
t1-p
からメッセージを読み取り、その値を JSON で取得します。kafka-protobuf-console-consumer --bootstrap-server localhost:9092 --from-beginning --topic t1-p --property schema.registry.url=http://localhost:8081
コンソールには次のように表示されます。
{"f1": "value1-p"}
ベースとなるデシリアライザーは Protobuf メッセージを作成し、それを読みやすいよう JSON 形式にシリアル化します。
別のシェルから curl コマンドを使用して、 Schema Registry に登録されているスキーマを調べます。
curl http://localhost:8081/subjects/t1-p-value/versions/1/schema
次のような結果が出力されます。
syntax = "proto3"; message MyRecord { string f1 = 1; }
次の curl コマンドを実行して、スキーマを詳しく確認します。さらに読みやすい出力結果を得るためには、パイプで jq に連結してください(
curl
のダウンロードメッセージは抑制します)。curl --silent -X GET http://localhost:8081/subjects/t1-p-value/versions/latest | jq
上のコマンドを実行すると、次のような結果が出力されます。
"subject": "t1-p-value", "version": 1, "id": 21, "schemaType": "PROTOBUF", "schema": "syntax = \"proto3\";\n\nmessage MyRecord {\n string f1 = 1;\n}\n"}
Confluent Control Center を使用して、スキーマとメッセージを調査します。
正常に生成されたメッセージは、Control Center (http://localhost:9021/)の Topics > <トピック名> > Messages にも表示されます。以前に送信されたメッセージを表示するには、パーティションを選択するか、タイムスタンプに移動しなければならない場合があります。(タイムスタンプに移動するには、数値を入力して Return キーを押します。デフォルトのパーティションは
1/Partition: 0
です。ここに示されているようなメッセージビューを取得するには、右上の カード アイコンを選択します。)作成するスキーマには、選択したトピックの Schemas タブからアクセスできます。
シャットダウンとクリーンアップのタスクを実行します。
- コンシューマーとプロデューサーを停止するには、それぞれのコマンドウィンドウで Ctrl キーを押しながら C キーを押します。
- Confluent Platform を停止するには、「
confluent local stop
」と入力します。 - 別のテストを最初から実行する前に既存のデータ(トピック、スキーマ、メッセージ)を削除する場合は、「
confluent local destroy
」と入力します。
Protobuf のスキーマ参照¶
Confluent Platform は、 スキーマ参照 (スキーマから別のスキーマを参照する機能)という概念をフルサポートします。
ちなみに
Confluent Cloud でも、Avro、Protobuf、JSON Schema の各形式でスキーマ参照がサポートされます。Confluent Cloud CLI から、ccloud schema-registry schema create の --refs <file>
フラグを使用することで他のスキーマを参照できます。
生成された Protobuf
クラスのインスタンスを KafkaProtobufSerializer
に渡せば、参照先のスキーマがすべて自動的に登録されます。
参照先のスキーマを手動で登録したければ、Schema Registry Maven プラグイン を使用するのが最も簡単です。または、REST API を使用して、参照先のスキーマを手動で登録することもできます。
この Protobuf の例では、order
サブジェクトとその参照(product
と customer
サブジェクト)のスキーマをインポートしています。
<plugin>
<groupId>io.confluent</groupId>
<artifactId>kafka-schema-registry-maven-plugin</artifactId>
<version>5.5.0</version>
<configuration>
<schemaRegistryUrls>
<param>http://192.168.99.100:8081</param>
</schemaRegistryUrls>
<subjects>
<order>src/main/protobuf/Order.proto</order>
<product>src/main/protobuf/Product.proto</product>
<customer>src/main/protobuf/Customer.proto</customer>
</subjects>
<schemaTypes>
<order>PROTOBUF</order>
<product>PROTOBUF</product>
<customer>PROTOBUF</customer>
</schemaTypes>
<references>
<order>
<reference>
<name>Product.proto</name>
<subject>product</subject>
</reference>
<reference>
<name>Customer.proto</name>
<subject>customer</subject>
</reference>
</order>
</references>
</configuration>
<goals>
<goal>register</goal>
</goals>
</plugin>
order
サブジェクトのスキーマは、おそらく次のようなものでしょう。
syntax = "proto3";
package io.confluent.examples.generated_sources.protobuf;
import "Product.proto";
import "Customer.proto";
message Order {
int32 order_id = 1;
string order_date = 2;
int32 order_amount = 3;
repeated Product products = 4;
Customer customer = 5;
}
そして、customer
サブジェクトのスキーマは、おそらく次のようなものでしょう。
syntax = "proto3";
package io.confluent.examples.generated_sources.protobuf;
message Customer {
int64 customer_id = 1;
string customer_name = 2;
string customer_email = 3;
string customer_address = 4;
}
後方互換性上の理由から、schemaType
と references
はどちらも省略可能です。schemaType
を省略した場合は、AVRO が指定されているものと見なされます。
同じトピック内の複数のイベントタイプ¶
スキーマ参照 が提供するのは、あるスキーマから他のスキーマを呼び出すしくみだけではありません。スキーマ参照を使用することで、複数のイベントタイプを同じトピックの中で効率よく組み合わせながらも、サブジェクト-トピック制約を維持することができます。
Protobuf でこれを実現するには、次のようにします。
デフォルトの サブジェクト命名方法 である
TopicNameStrategy
を使用します。この命名方法では、スキーマのルックアップに使用されるサブジェクトがトピック名を使用して決定されます。また、この命名方法によって、サブジェクト-トピック制約の適用が促されます。メッセージで
oneOf
をラップしてスキーマ参照のリストを定義します。その例を次に示します。syntax = "proto3"; package io.confluent.examples.proto; import "Customer.proto"; import "Product.proto"; import "Order.proto"; message AllTypes { oneof oneof_type { Customer customer = 1; Product product = 2; Order order = 3; } }
スキーマが登録されるときに、参照バージョンの配列を送信します。その例を次に示します。
[ { "name": "Customer.proto", "subject": "customer", "version": 1 }, { "name": "Product.proto", "subject": "product", "version": 1 }, { "name": "Order.proto", "subject": "order", "version": 1 } ]
ちなみに
- Protobuf では、トップレベルの
oneOf
が許容されません。メッセージでoneOf
をラップする必要があるのはそのためです。 - メッセージで
oneOf
をラップする利点は、トップレベルスキーマの自動登録がデフォルトで適切に機能することです(同じユースケースでも、 Avro や JSON Schema では、別途設定が必要になります)。また、Protobuf の場合、参照先スキーマもすべて再帰的に自動登録されます。
- Protobuf では、トップレベルの
ラッパーでの null 値の処理¶
Protobuf では、オブジェクトフィールドに null 値を設定することはできません。null 値を使用すると null ポインターの例外が発生します。
回避策として、google/protobuf/wrappers.proto
をインポートし、wrapper.for.nullables
コンバーターを使用することができます。これにより null 許容フィールドに Protobuf ラッパーの型を使用するかどうかを指定できます。このブール値のデフォルトは false
です。true
に設定すると、null 許容フィールドで GitHub の protocolbuffers/protobuf、および google.protobuf package documentation に記載されているラッパーの型が使用されます。
たとえば、null 値を許容しない名前でデフォルト値が空の文字列(""
)の場合、
message GreetRequest {
string name = 1;
}
wrappers.proto
のラッパークラス(この例では StringValue)を使用して、null を許容する名前に置き換えることができます。
message GreetRequest {
google.protobuf.StringValue name = 1;
}
Confluent Platform では、Protobuf コンバーターでフラグ wrappers.for.nullable
を使用すると、次のような結果になります。
- Connect スキーマから Protobuf スキーマへの変換では、オプションの Connect フィールドがラッパークラスを使用する Protobuf フィールドに変換されます。
- Protobuf スキーマから Connect スキーマへの変換では、ラッパークラスを使用する Protobuf フィールドがオプションの Connect フィールドに変換されます。
詳細については、Tomas Basham の記事「Protocol Buffers and Optional Fields」および Protocol Buffers 開発者ガイドの「JSON Mapping」にある表の「Wrapper types」の null 値の説明を参照してください。
Protobuf スキーマの互換性ルール¶
スキーマ進化 は互換性ルールによって支えられています。また下流のコンシューマーは、互換性ルールのおかげで、古いスキーマと新しいスキーマでエンコードされたデータを処理することができます。特に Protobuf と Avro に言えることですが、これらのルールは異なる形式間である程度共通しています。ただし Protobuf の後方互換性は例外で、この点は両者間で異なっています。
Protobuf の互換性ルール(後方互換性を含む)について詳しくは、概要の「互換性チェック」を参照してください。
おすすめの記事¶
- Kafka クライアントのプロデューサーとコンシューマーの例が「Apache Kafka® のサンプルコード」に掲載されています(Avro を使用している例と使用していない例があります)。
- Schema Registry API リファレンス
- Schema Registry API の使用例
- Confluent Cloud におけるスキーマ管理のクイックスタート
- Protocol Buffers (Protobuf) Developer's Guide
- ブログの投稿記事: Getting Started with Protobuf in Confluent Cloud
- ブログ記事: 「Understanding Protobuf Compatibility」