Kafkaプロトコル ガイド

このドキュメントはKafkaで実装されている wire プロトコルを対象とします。利用可能なリクエスト、それらのバイナリの形式、そしてクライアントを実装するためにそれらを利用する適切な方法をカバーするプロトコルの読み易いガイドを与えるつもりです。このドキュメントではここで説明されている基本的な設計と用語を理解していると仮定します。

下準備

ネットワーク

Kafka はTCP上でバイナリのプロトコルを使用します。プロトコルはリクエスト リスポンス メッセージのペアとして全ての API を定義します。全てのメッセージはサイズが決められていて、以下のプリミティブな型からできています。

クライアントはソケットを初期化し、その後リクエストメッセージの順列を書き込み、対応する応答メッセージを読み返します。接続時あるいは切断時にハンドシェイクは必要ありません。TCP ハンドシェイクのコストを償却するために多くのリクエストに永続化接続を維持する場合は、TCP はより幸せになれますが、このペナルティを超えると接続はかなり安価になります。

データはパーティション化され、クライアントはデータを持つサーバに話しかける必要があるため、クライアントは複数のブローカーへの接続を維持する必要がないでしょう。しかし、一般的に一つのクライアントインスタンスから一つのブローカーへの複数の接続を維持する必要はありません(つまり、接続プール)。

サーバは1つのTCP接続上でリクエストが送信された順番で処理され、応答も同様に返されるだろうことを保証します。ブローカーのリクエストの処理はこの順番を保証するために接続ごとに1つの送信中のリクエストのみを許可します。クライアントはリクエストのパイプラインを実装し高パフォーマンスを達成するために非ブロッキング IOを使うことができます(そして理想的にはそうすべきです)。つまり、発行されたリクエストは背後のOSのソケットバッファにバッファされるため、先行するリクエストの応答を待つ間でさえリクエストを送信することができます。全てのリクエストはクライアントによって起動され、注意された箇所を除きサーバから対応する応答のメッセージを結果として得ます。

サーバはリクエストサイズ上の設定可能な最大制限を持ち、この制限を超えた全てのリクエストは結果的に切断されるソケットになるでしょう。

パーティショニングとブートストラッピング

Kafkaはパーティション化されたシステムのため、全てのサーバが完全なデータセットを持つわけではありません。代わりに、トピックが事前定義された数のパーティションに分割され、各パーティションが何らかのレプリケーション係数 N でレプリケートされることを思い出してください。トピックパーティション自体は単純に 0, 1, ..., P-1 の番号が付いた "commit logs" です。

この性質の全てのシステムには、特定のデータが特定のパーティションにどのように割り当てられるかという問題があります。Kafkaクライアントは直接この割り当てを制御し、ブローカー自身はどのメッセージが特定のパーティションに発行されるべきかの特定のセマンティクスを強制しません。むしろ、メッセージを発行するために、クライアントはメッセージを特定のパーティションに直接指定し、メッセージをフェッチする時に特定のパーティションからフェッチします。もし2つのクライアントが同じパーティション スキーマを使う場合、それらはキーからパーティションへのマッピングを計算するために同じメソッドを使うべきです。

データを発行あるいは取得するためのこれらのリクエストは現在のところ指定されたパーティションのためのリーダーとして振舞うブローカーに送信されるべきです。この条件はブローカーによって強制され、そのため間違ったブローカーへの特定のパーティションのためのリクエストは NotLeaderForPartition エラーコードという結果に終わるでしょう (以下で説明されます)。

クライアントが正しいホストにリクエストを指示することができるように、どのトピックが存在するか、どのパーティションを持っているか、そしてどのブローカーが現在のところそれらのパーティションをホストするかを、クライアントはどうやって見つけることができますか?この情報は動的なため、なんらかの静的なマッピングファイルを使って各クライアントを単純に設定することはできません。代わりに全てのKafkaブローカーはクラスタの現在の状態を記述するメタデータのリクエストを答えることができます: 何のトピックがあるか、どのパーティションをそれらのトピックが持つか、どのブローカーがそれらのパーティションのためのリーダーか、そしてこれらのブローカーのためのホストおよびポートの情報。

別の言い方をすると、クライアントはどうにかして1つのブローカーを見つけ、そのブローカーは存在する他のブローカーとどのパーティションをホストするかについてクライアントにこたえるでしょう。この最初のブローカー自身がダウンするかもしれないため、クライアント実装のベストプラクティスは、2つまたは3つのURLを取得してブートストラップすることです。ユーザはロードバランサを使うか、まさに静的にクライアント中の2あるいは3のkafkaホストを設定するかを選択することができます。

クライアントはクラスタが変更されたかどうかを確認するためにポーリングし続ける必要はありません; メタデータが期限切れであることを示すエラーを受け取るまで、メタデータをインスタンス化する時に一度メタデータをフェッチすることができます。このエラーには2つの形式があります: (1) クライアントが特定のブローカーと通信できないことを示すソケットエラー。(2) このブローカーはリクエストされたデータをホストしていないことを示す、リクエストへの応答エラーコード。

  1. 接続可能なURLが見つかるまで、"bootstrap" Kafka URLのリストを循環します。クラスタのメタデータを取得する。
  2. フェッチを処理あるいはリクエストを生成し、送信先あるいは取得先のトピック/パーティションに基づいて適切なブローカーに要求を送信します。
  3. もし適切なエラーを受け取ると、メタデータをリフレッシュし再び試します。
パーティショニングの戦略

上述したように、メッセージの割り当ては生成しているクライアントが制御します。とはいえ、この機能はエンドユーザにどのように公開すべきですか?

パーティション化はKafkaで実際に2つの目的を提供します:

  1. データとリクエストの負荷をブローカー上でバランスします
  2. ローカルの状態を許可し、パーティション内の順番を維持しながら、コンシューマプロセス間で処理を分割する方法として機能します。このセマンティックをパーティショニングと呼びます。

指定されたユースケースについて、これらのうちの1つだけあるいは両方について心配するかもしれません。

単純なロードバランスを実現するための単純な方法はクライアントが単に全てのブローカーに渡ってリクエストをラウンドロビンすることです。別のやり方としては、ブローカーより多くのプロデューサがある環境では、各クライアントがランダムに1つのパーティションを選びそれに発行します。この後者の戦略は遥かに少ないTCP接続という結果になるでしょう。

セマンティック パーティショニングはメッセージをパーティションへ割り当てるためにメッセージ内の幾つかのキーを使うことを意味します。例えば、クリックメッセージストリームを処理している場合、特定のユーザのための全てのデータgーあ1つのコンシューマに送られるように、ユーザIDでストリームを分割したいかもしれません。これを実現するために、クライアントはメッセージに関係するキーを取り、メッセージを配送するパーティションを選択するためにこのキーの何らかのハッシュを使用します。

バッチ

APIは効率化のために小さなものをまとめてバッチ処理することを推奨しています。これは非常に重要なパフォーマンスの向上であることが分かりました。メッセージを送信するAPIとメッセージを取得するためのAPIのどちらもが、これを促進する1つのメッセージではなく常に一連のメッセージで動作します。賢いクライアントはこれを使うことができ、個別に送信されるメッセージをまとめて一括して送信する "非同期" モードをサポートします。これをさらに進めて、複数のトピックとパターンにまたがるバッチ処理を許可します。つまり、生成リクエストは多くのパーティションに追加するデータを含み、取得リクエストは多くのパーティションからデータを一度に取り出すかもしれません。

クライアントの実装者はそうしたければこれを無視し、全てを一度に送信することを選択することができます。

互換性

Kafka には、"双方向性" クライアント互換性ポリシーがあります。つまり、新しいクライアントは古いサーバと通信でき、古いクライアントは新しいサーバと通信できます。これにより、ユーザはダウンタイムを発生させることなくクライアントまたはサーバをアップグレードすることができます。

Kafka プロトコルは時間と共に変化したため、クライアントとサーバはネットワークを介して送信するメッセージのスキーマについて同意する必要があります。これは API のバージョンによって行われます。

各リクエストが送信される前に、クライアントは API キーと API バージョンを送信します。これら2つの16ビットを一緒に使うと、従うメッセージのスキーマをユニークに識別します。

クライアントが様々なAPIのバージョンをサポートすることを意図しています。特定のブローカーと通信する場合、指定されたクライアントは両方でサポートされる最も高いAPIバージョンを使い、リクエスト中のこのバージョンを示す必要があります。

サーバはサポートしないバージョンのリクエストを拒否し、リクエスト内に含まれるバージョンに基づいて期待されるプロトコルを使って常に応答します。アップグレードパスの目的は、新しい機能が最初に(古いクライアントがそれらを使用しない)サーバに展開され、その後新しいクライアントが配備されると新しい機能が次第に活用されることです。

KIP-482 タグ付きフィールドはバージョン番号を増やさずにリクエストに追加できることに注意してください。これにより、互換性を損なうことなくメッセージスキーマを進化させる追加の方法が提供されます。タグ付きのフィールドは、フィールドが設定されていない場合、スペースを取りません。したがって、フィールドがほとんど使われない場合は、必須のスキーマに配置するよりタグ付きフィールドにする方が効率的です。ただし、タグ付きフィールドは、それを知らない受信者によって無視されるため、送信者が望んでいない動作の場合は問題が生じる可能性があります。そのような場合は、バージョンのバンプがより適切な場合があります。

サポートされるAPIバージョンの取得

複数のブローカーのバージョンに対して動作するために、クライアントはブローカーが様々なAPIのどのバージョンをサポートするかを知る必要があります。KIP-35で説明されているように、ブローカーは 0.10.0.0からこの情報を公開します。クライアントは、クライアントとブローカーの両方でサポートされる最も高いAPIバージョンを選択するために、サポートされるAPIバージョンの情報を使わなければなりません。そのようなバージョンが無い場合は、エラーがユーザに報告される必要があります。

以下のシーケンスが、サポートされるAPIバージョンをブローカーから取得するためにクライアントによって使われるかもしれません。

  1. クライアントは、ブローカーとの接続が確立した後で、ApiVersionsRequest をブローカーに送信します。SSLが有効な場合は、これはSSL接続が確立した後で起こります。
  2. ApiVersionsRequest を受信する時、ブローカーは現在の承認状態に関係なく、サポートされる ApiKey とバージョンの完全なリストを返します (例えば、SASL リスナーでの SASL 認証の前に、SSL ハンドシェイクが完了するまで、SSL リスナーで Kafka プロトコルリクエストが発生しないことに注意してください)。これがブローカーのバージョンに関する情報を漏らすと考える場合、回避策はApiVersionRequest が利用できない接続の初期段階で実行されるクライアント認証でSSLを使うことです。また、0.10.0.0 より古いブローカーバージョンはこのAPIをサポートしていないため、リクエストを無視するか、リクエストへの応答で接続を閉じることに注意してください。
  3. APIの複数のバージョンがブローカーとクライアントでサポートされる場合は、クライアントはブローカーとクライアント自身でサポートされる最新のバージョンを使うことをお勧めします。
  4. プロトコルのバージョンの廃止は、プロトコルのドキュメントの中でAPIバージョンを廃止予定としてマークすることで行われます。
  5. ブローカーから取得されるサポートされるAPIのバージョンは、情報が取得された接続についてのみ有効です。切断された場合、ブローカーがその間にアップグレード/ダウングレードしたかもしれないため、クライアントはブローカーから再度情報を取得する必要があります。
SASL認証シーケンス

以下のシーケンスがSASL認証のために使われます:

  1. ブローカーによってサポートされるリクエストのバージョン範囲を取得するために、Kafka ApiVersionsRequest が送信されるかもしれません。これはオプションです。
  2. 認証のためのSASLの仕組みを含むKafka SaslHandshakeRequest はクライアントによって送信されます。サーバでメカニズムが有効になっている場合、サーバはサポートされるメカニズムのリストを使って応答し、クライアントの接続を閉じます。もしメカニズムがサーバで有効になっている場合、サーバは正常な王乙を送信し、SASL認証を続行します。
  3. これで、実際のSASL認証が行われます。もしSaslHandshakeRequestバージョンが0の場合、メカニズムに対応する一連のSASLクライアントおよびサーバ トークンは、Kafkaプロトコル ヘッダでメッセージをラップすることなく、曖昧なパケットとして送信されます。もし SaslHandshakeRequest バージョンが1の場合、実際のSASLトークンがKafkaプロトコルでラップされたSaslAuthenticate リクエスト/応答が使われます。ブローカーからの最終的なメッセージでのエラーコードは、認証が成功あるいは失敗したかを示します。
  4. もし認証が成功した場合、後続のパケットはKafka APIリクエストとして処理されます。それ以外の場合、クライアントの接続が閉じられます。

0.9.0.x のクライアントとの相互運用性のために、サーバによって受信された最初のパケットは、有効なKafkaリクエストでない場合、SASL/GSSAPI クライアント トークンとして処理されます。SASL/GSSAPI 認証は上記の最初の2つのステップをスキップして、このパケットから実行されます。

プロトコル

プロトコルのプリミティブな型

プロトコルは以下のプリミティブ型から構築されます。

種類 説明
BOOLEANバイトでのboolean値を表します。値 0 および 1 はそれぞれfalseおよびtrueを表すために使われます。boolean値を読む時に、非0の値はtrueと見なされます。
INT8-27 から 27-1 までの整数を表します。
INT16-215 から 215-1 までの整数を表します。値はネットワークバイト順(ビッグ エンディアン)の2バイトを使ってエンコードされます。
INT32-231 から 231-1 までの整数を表します。値はネットワークバイト順(ビッグ エンディアン)の4バイトを使ってエンコードされます。
INT64-263 から 263-1 までの整数を表します。値はネットワークバイト順(ビッグ エンディアン)の8バイトを使ってエンコードされます。
UINT320 から 232-1 までの整数を表します。値はネットワークバイト順(ビッグ エンディアン)の4バイトを使ってエンコードされます。
VARINT-231 から 231-1 までの整数を表します。エンコーディングは Google Protocol Buffersの可変長zig-zagエンコーディングに従います。
VARLONG-263 から 263-1 までの整数を表します。エンコーディングは Google Protocol Buffersの可変長zig-zagエンコーディングに従います。
UUIDRepresents a java.util.UUID. 値はネットワークバイト順(ビッグ エンディアン)の16バイトを使ってエンコードされます。
FLOAT64倍精度64ビット形式の IEEE 754 の値を表します。値はネットワークバイト順(ビッグ エンディアン)の8バイトを使ってエンコードされます。
STRING文字のシーケンスを表します。最初に長さNがINT16として指定されます。次に文字シーケンスのUTF-8エンコーディングであるNバイトが続きます。長さは負であってはなりません。
COMPACT_STRING文字のシーケンスを表します。最初に、長さ N + 1 が UNSIGNED_VARINT として指定されます。次に文字シーケンスのUTF-8エンコーディングであるNバイトが続きます。
NULLABLE_STRING文字のシーケンスあるいはnullを表します。非null文字列については、最初に長さNがINT16として指定されます。次に文字シーケンスのUTF-8エンコーディングであるNバイトが続きます。null値は長さ-1でエンコードされ、後続のバイトはありません。
COMPACT_NULLABLE_STRING文字のシーケンスを表します。最初に、長さ N + 1 が UNSIGNED_VARINT として指定されます。次に文字シーケンスのUTF-8エンコーディングであるNバイトが続きます。null 文字列は長さ 0 で表されます。
BYTESバイトの生のシーケンスを表します。最初に長さNがINT32として指定されます。次にNバイトが続きます。
COMPACT_BYTESバイトの生のシーケンスを表します。最初に、長さ N + 1 が UNSIGNED_VARINT として指定されます。N バイトが続きます。
NULLABLE_BYTESバイトの生のシーケンスあるいはnullを表します。非nullの値については、最初に長さNがINT32として指定されます。次にNバイトが続きます。null値は長さ-1でエンコードされ、後続のバイトはありません。
COMPACT_NULLABLE_BYTESバイトの生のシーケンスを表します。最初に、長さ N + 1 が UNSIGNED_VARINT として指定されます。N バイトが続きます。null オブジェクトは長さ 0 で表されます。
RECORDS一連のKafkaレコードを NULLABLE_BYTES として表します。レコードの詳細な説明については、Message Setsを見てください。
ARRAY指定された型Tのオブジェクトのシーケンスを表します。型Tはプリミティブ型(例えば文字列)あるいは構造体のどちらかです。最初に長さNがINT32として指定されます。次に型TのN個のインスタンスが続きます。null配列は長さ-1で表されます。プロトコルのドキュメントでは、Tインスタンスの配列は [T] として参照されます。
COMPACT_ARRAY指定された型Tのオブジェクトのシーケンスを表します。型Tはプリミティブ型(例えば文字列)あるいは構造体のどちらかです。最初に、長さ N + 1 が UNSIGNED_VARINT として指定されます。次に型TのN個のインスタンスが続きます。null配列は長さ-0で表されます。プロトコルのドキュメントでは、Tインスタンスの配列は [T] として参照されます。
リクエストの書式の構文を読む際の注意

以下のBNF はリクエストと応答のバイナリ形式の正確な文脈自由文法を提供します。BNFは人間が読める名前を提供するために意図的にコンパクトではありません。BNFの場合と同様に、一連の生成物は連結を示します。複数の可能な生成物がある場合、これらは '|' で区切られ、グループ化のために括弧で囲むことができます。最上位の定義は常に最初に指定され、後続の子パートはインデントされます。

一般的なリクエストと応答の構造

全てのリクエストと応答は、このドキュメントの残りの部分で徐々に説明される以下の文法から生成されます。

RequestOrResponse => Size (RequestMessage | ResponseMessage)
  Size => int32
フィールド説明
message_sizemessage_size フィールドは、後続のリクエストあるいは応答メッセージのサイズをバイト単位で指定します。クライアントは、最初にこの4バイトのサイズを整数Nとして読み取り、次に要求の後続のNバイトを読み取って解析することにより、リクエストを読み取ることができます。
レコードバッチ

レコードバッチ形式の説明はここで見つけることができます。

定数

エラーコード

数値コードを使ってサーバで発生した問題を示します。これらは、クライアントによって例外またはクライアント言語の適切なエラー処理メカニズムによって変換することができます。現在使用中のエラーコードの表は以下の通りです:

エラー コード 再試行可能 説明
UNKNOWN_SERVER_ERROR-1Falseリクエストを処理する時にサーバで予期しないエラーが発生しました。
NONE0False
OFFSET_OUT_OF_RANGE1Falseリクエストされたオフセットは、サーバによって維持されるオフセットの範囲内に無い。
CORRUPT_MESSAGE2TrueこのメッセージはCRC checksumに失敗、有効なサイズを超過、圧縮されたトピックのnullキーを持つか、そうでなければ衝突した。
UNKNOWN_TOPIC_OR_PARTITION3Trueこのサーバはこのトピック-パーティションをホストしません。
INVALID_FETCH_SIZE4Falseリクエストされたフェッチサイズは無効です。
LEADER_NOT_AVAILABLE5Trueリーダーシップの選出の最中にこのトピック-パーティションのためのリーダーが無い。
NOT_LEADER_OR_FOLLOWER6Trueリーダーのみを対象としたリクエストの場合、このエラーはブローカーが現在のリーダーではないことを示します。レプリカを対象としたリクエストの場合、このエラーはブローカーがトピックパーティションのレプリカではないことを示します。
REQUEST_TIMED_OUT7Trueリクエストがタイムアウトした。
BROKER_NOT_AVAILABLE8Falseブローカーが利用可能では無い。
REPLICA_NOT_AVAILABLE9Trueリクエストされたトピック-パーティションに関してレプリカが利用可能では無い。ブローカーがトピックパーティションのレプリカではない場合、生成/フェッチリクエストおよびリーダーまたはフォロワーのみを対象とした他のリクエストは NOT_LEADER_OR_FOLLOWER を返します。
MESSAGE_TOO_LARGE10Falseリクエストはサーバが受け付ける最大のメッセージサイズより大きなメッセージを含んでいた。
STALE_CONTROLLER_EPOCH11Falseコントローラが他のブローカーに移動した。
OFFSET_METADATA_TOO_LARGE12Falseオフセット リクエストのメタデータのフィールドがあまりに大きかった。
NETWORK_EXCEPTION13True応答が受け取られる前にサーバが切断された。
COORDINATOR_LOAD_IN_PROGRESS14Trueコーディネーターが呼び出し中でリクエストを処理できない。
COORDINATOR_NOT_AVAILABLE15Trueコーディネータが利用可能ではない。
NOT_COORDINATOR16Trueこれは現在のコーディネータではない。
INVALID_TOPIC_EXCEPTION17Falseリクエストが無効なトピック上でオペレーションを実施しようとした。
RECORD_LIST_TOO_LARGE18Falseサーバ上で設定されたセグメントのサイズより大きなメッセージバッチに含まれるリクエスト。 on the server.
NOT_ENOUGH_REPLICAS19True必要とされるより少ない同期中のレプリカがあるためメッセージは拒絶されます。
NOT_ENOUGH_REPLICAS_AFTER_APPEND20Trueメッセージはログに書き込まれますが、必要とされるより同期中のレプリカが少ないです。
INVALID_REQUIRED_ACKS21FalseProduce request specified an invalid value for required acks.
ILLEGAL_GENERATION22False指定されたグループ生成idが無効です。
INCONSISTENT_GROUP_PROTOCOL23Falseプロトコルによってサポートされるグループのメンバーが既存のメンバーと互換性が無いか、最初のグループメンバーが空のプロトコル型あるいは空のプロトコルリストを使って参加しようとしました。
INVALID_GROUP_ID24False設定されたグループIdが無効です
UNKNOWN_MEMBER_ID25Falseコーディネーターはこのメンバーに気づいていません。
INVALID_SESSION_TIMEOUT26Falseセッションタイムアウトがブローカーによって許可される範囲内にありません (group.min.session.timeout.ms と group.max.session.timeout.ms で設定される)。
REBALANCE_IN_PROGRESS27Falseグループはリバランシングしているため、再参加が必要です。
INVALID_COMMIT_OFFSET_SIZE28Falseコミットしているオフセットデータのサイズが有効ではない。
TOPIC_AUTHORIZATION_FAILED29Falseトピックの認証が失敗しました。
GROUP_AUTHORIZATION_FAILED30Falseグループ認証が失敗しました。
CLUSTER_AUTHORIZATION_FAILED31Falseクラスタの認証が失敗しました。
INVALID_TIMESTAMP32Falseメッセージのタイムスタンプが許容範囲を超えました。
UNSUPPORTED_SASL_MECHANISM33FalseブローカーはリクエストされたSASLの仕組みをサポートしません。
ILLEGAL_SASL_STATE34False現在の SASL 状態では、リクエストは無効です。
UNSUPPORTED_VERSION35FalseAPIのバージョンがサポートされません。
TOPIC_ALREADY_EXISTS36Falseこの名前を持つトピックが既に存在します。
INVALID_PARTITIONS37Falseパーティションの数が1未満である。
INVALID_REPLICATION_FACTOR38Falseリプリケーションの係数が1未満、あるいは利用可能なブローカーの数よりも大きい。
INVALID_REPLICA_ASSIGNMENT39Falseレプリカの割り当てが無効です。
INVALID_CONFIG40False設定が無効です。
NOT_CONTROLLER41Trueこのクラスタの正しいコントローラではありません。
INVALID_REQUEST42Falseこれは、クライアントライブラリによってリクエストの形式が間違っているか、メッセージが互換性のないブローカーに送信されたことが原因であると考えられます。詳細はブローカーのログを見てください。
UNSUPPORTED_FOR_MESSAGE_FORMAT43Falseブローカー上のメッセージ形式のバージョンはリクエストをサポートしません。
POLICY_VIOLATION44Falseリクエストのパラメータは設定されたポリシーを満たしません。
OUT_OF_ORDER_SEQUENCE_NUMBER45Falseブローカーは順番がバラバラのシーケンス番号を受け取りました
DUPLICATE_SEQUENCE_NUMBER46Falseブローカーは重複したシーケンス番号を受け取りました
INVALID_PRODUCER_EPOCH47FalseProducer attempted to produce with an old epoch.
INVALID_TXN_STATE48Falseプロデューサは無効な状態でトランザクション操作を試みました。
INVALID_PRODUCER_ID_MAPPING49Falseプロデューサは現在トランザクションidに割り当てられていないプロデューサidを使おうと試みました。
INVALID_TRANSACTION_TIMEOUT50Falseトランザクションのタイムアウトはブローカーによって許可される最大の値(transaction.max.timeout.msによって設定される)よりも大きいです。
CONCURRENT_TRANSACTIONS51Falseプロデューサは同じトランザクションで実行中の他の同時に起こるオペレーションの間に、トランザクションを更新しようとしました。
TRANSACTION_COORDINATOR_FENCED52FalseWriteTxnMakerを送信するトランザクションのコーディネータが指定されたプロデューサのための現在のコーディネーターではないことを示す。
TRANSACTIONAL_ID_AUTHORIZATION_FAILED53Falseトランザクション Idの認証が失敗した。
SECURITY_DISABLED54Falseセキュリティ機能が無効である。
OPERATION_NOT_ATTEMPTED55Falseブローカーはこのオペレーションを実行しようとしなかった。これはバッチの一部の操作が失敗したバッチRPCで発生する可能性があり、ブローカーは残りを試行せずに応答します。
KAFKA_STORAGE_ERROR56Trueディスク上のログへアクセスしようとした時のディスクエラー。
LOG_DIR_NOT_FOUND57Falseユーザ定義のログディレクトリがブローカーの設定の中で見つからない。
SASL_AUTHENTICATION_FAILED58FalseSASL 認証が失敗した。
UNKNOWN_PRODUCER_ID59Falseブローカーが質問中のプロデューサIDに関係するプロデューサのメタデータの場所を見つけることができなかった時に、ブローカーによってこの例外が上げられます。例えばプロデューサのレコードの維持期間が過ぎたためにそれらが削除された場合に、これが起きるかもしれません。プロデューサIdの最後のレコードが削除されると、プロデューサのメタデータはブローカーから削除され、プロデューサによる以降の追加はこの例外を返します。
REASSIGNMENT_IN_PROGRESS60Falseパーティションの割り当てが実行中。
DELEGATION_TOKEN_AUTH_DISABLED61False移譲トークンの機能が有効ではありません。
DELEGATION_TOKEN_NOT_FOUND62False移譲トークンがサーバ上で見つかりません。
DELEGATION_TOKEN_OWNER_MISMATCH63False指定されたプリンシパルは有効な Owner/Renewer ではありません。
DELEGATION_TOKEN_REQUEST_NOT_ALLOWED64False移譲トークンのリクエストは、PLAINTEXT/1-way SSL チャネルおよび以上トークン認証チャネル上では許可されません。
DELEGATION_TOKEN_AUTHORIZATION_FAILED65False移譲トークンの認証が失敗しました。
DELEGATION_TOKEN_EXPIRED66False移譲トークンが期限切れになりました。
INVALID_PRINCIPAL_TYPE67False提供されたプリンシパルの型はサポートされません。
NON_EMPTY_GROUP68Falseグループは空ではありません。
GROUP_ID_NOT_FOUND69Falseグループidが存在しません。
FETCH_SESSION_ID_NOT_FOUND70Trueフェッチ セッションIDが見つかりませんでした。
INVALID_FETCH_SESSION_EPOCH71Trueフェッチ セッションのエポックが無効です。
LISTENER_NOT_FOUND72Trueリーダーブローカーには、メタデータのリクエストが処理されたリスナーに一致するリスナーはありません。
TOPIC_DELETION_DISABLED73Falseトピックの削除は無効です。
FENCED_LEADER_EPOCH74Trueリクエスト内のリーダーエポックは、ブローカー上のエポックよりも古いです。
UNKNOWN_LEADER_EPOCH75Trueリクエスト内のリーダーエポックは、ブローカー上のエポックよりも新しいです。
UNSUPPORTED_COMPRESSION_TYPE76Falseリクエストしているクライアントは指定されたパーティションの圧縮型をサポートしません。
STALE_BROKER_EPOCH77Falseブローカーのエポックが変更されました。
OFFSET_NOT_AVAILABLE78Trueリーダーの最高のウォーターマークは最近のリーダー選出から追いついていないため、オフセットが単調に増加することは保証できません。
MEMBER_ID_REQUIRED79Falseグループメンバーは実際にコンシューマグループに入る前に有効なメンバー id を持つ必要があります。
PREFERRED_LEADER_NOT_AVAILABLE80True優先リーダーは利用できませんでした。
GROUP_MAX_SIZE_REACHED81Falseコンシューマグループが最大サイズに達しました。
FENCED_INSTANCE_ID82False同じ group.instance.idを持つ他のコンシューマが異なるmember.idで登録されているために、ブローカーはこの静的なコンシューマを拒否しました。
ELIGIBLE_LEADERS_NOT_AVAILABLE83True的確なパーティションリーダーは利用できませんでした。
ELECTION_NOT_NEEDED84Trueトピックパーティションにはリーダー選出は必要ありません。
NO_REASSIGNMENT_IN_PROGRESS85Falseパーティションの再割り当ては行われていません。
GROUP_SUBSCRIBED_TO_TOPIC86Falseコンシューマグループがアクティブに購読している間は、トピックのオフセットの削除は禁止されています。
INVALID_RECORD87Falseこのレコードはブローカーでの検証に失敗したため、拒否されます。
UNSTABLE_OFFSET_COMMIT88Trueクリアする必要がある不安定なオフセットがあります。
THROTTLING_QUOTA_EXCEEDED89TrueThe throttling quota has been exceeded.
PRODUCER_FENCED90FalseThere is a newer producer with the same transactionalId which fences the current one.
RESOURCE_NOT_FOUND91FalseA request illegally referred to a resource that does not exist.
DUPLICATE_RESOURCE92FalseA request illegally referred to the same resource twice.
UNACCEPTABLE_CREDENTIAL93FalseRequested credential would not meet criteria for acceptability.
INCONSISTENT_VOTER_SET94FalseIndicates that the either the sender or recipient of a voter-only request is not one of the expected voters
INVALID_UPDATE_VERSION95FalseThe given update version was invalid.
FEATURE_UPDATE_FAILED96FalseUnable to update finalized features due to an unexpected server error.
Api キー

以下はリクエスト内のApiKeyが以下の各リクエスト型に注意することができる数値コードです。

名前 キー
Produce0
Fetch1
ListOffsets2
Metadata3
LeaderAndIsr4
StopReplica5
UpdateMetadata6
ControlledShutdown7
OffsetCommit8
OffsetFetch9
FindCoordinator10
JoinGroup11
Heartbeat12
LeaveGroup13
SyncGroup14
DescribeGroups15
ListGroups16
SaslHandshake17
ApiVersions18
CreateTopics19
DeleteTopics20
DeleteRecords21
InitProducerId22
OffsetForLeaderEpoch23
AddPartitionsToTxn24
AddOffsetsToTxn25
EndTxn26
WriteTxnMarkers27
TxnOffsetCommit28
DescribeAcls29
CreateAcls30
DeleteAcls31
DescribeConfigs32
AlterConfigs33
AlterReplicaLogDirs34
DescribeLogDirs35
SaslAuthenticate36
CreatePartitions37
CreateDelegationToken38
RenewDelegationToken39
ExpireDelegationToken40
DescribeDelegationToken41
DeleteGroups42
ElectLeaders43
IncrementalAlterConfigs44
AlterPartitionReassignments45
ListPartitionReassignments46
OffsetDelete47
DescribeClientQuotas48
AlterClientQuotas49
DescribeUserScramCredentials50
AlterUserScramCredentials51
AlterIsr56
UpdateFeatures57

メッセージ

この章は個々のAPIメッセージ、それらの使い方、それらのバイナリ形式、およびそれらのフィールドの意味の詳細を与えます。

Headers:
Request Header v0 => request_api_key request_api_version correlation_id 
  request_api_key => INT16
  request_api_version => INT16
  correlation_id => INT32
フィールド 説明
request_api_keyこのリクエストの API キー。
request_api_versionこのリクエストの API バージョン。
correlation_idこのリクエストの相関 ID。
Request Header v1 => request_api_key request_api_version correlation_id client_id 
  request_api_key => INT16
  request_api_version => INT16
  correlation_id => INT32
  client_id => NULLABLE_STRING
フィールド 説明
request_api_keyこのリクエストの API キー。
request_api_versionこのリクエストの API バージョン。
correlation_idこのリクエストの相関 ID。
client_idクライアント ID 文字列。
Request Header v2 => request_api_key request_api_version correlation_id client_id TAG_BUFFER 
  request_api_key => INT16
  request_api_version => INT16
  correlation_id => INT32
  client_id => NULLABLE_STRING
フィールド 説明
request_api_keyこのリクエストの API キー。
request_api_versionこのリクエストの API バージョン。
correlation_idこのリクエストの相関 ID。
client_idクライアント ID 文字列。
_tagged_fieldsタグ付きのフィールド
Response Header v0 => correlation_id 
  correlation_id => INT32
フィールド 説明
correlation_idこの応答の相関 ID。
Response Header v1 => correlation_id TAG_BUFFER 
  correlation_id => INT32
フィールド 説明
correlation_idこの応答の相関 ID。
_tagged_fieldsタグ付きのフィールド
Produce API (Key: 0):
Requests:

Produce Request (Version: 0) => acks timeout [topic_data] 
  acks => INT16
  timeout => INT32
  topic_data => topic [data] 
    topic => STRING
    data => partition record_set 
      partition => INT32
      record_set => RECORDS
フィールド 説明
acksリクエストが完了したと見なす前にリーダーが受け取ることをプロデューサが要求する通知の数。可能な値: 非通知については0、リーダーのみ 1, 完全なISRについては -1。
timeout応答を待つ時間のミリ秒。
topic_datanull
トピックトピック名
datanull
パーティショントピックのパーティションid
record_setnull

Produce Request (Version: 1) => acks timeout [topic_data] 
  acks => INT16
  timeout => INT32
  topic_data => topic [data] 
    topic => STRING
    data => partition record_set 
      partition => INT32
      record_set => RECORDS
フィールド 説明
acksリクエストが完了したと見なす前にリーダーが受け取ることをプロデューサが要求する通知の数。可能な値: 非通知については0、リーダーのみ 1, 完全なISRについては -1。
timeout応答を待つ時間のミリ秒。
topic_datanull
トピックトピック名
datanull
パーティショントピックのパーティションid
record_setnull

Produce Request (Version: 2) => acks timeout [topic_data] 
  acks => INT16
  timeout => INT32
  topic_data => topic [data] 
    topic => STRING
    data => partition record_set 
      partition => INT32
      record_set => RECORDS
フィールド 説明
acksリクエストが完了したと見なす前にリーダーが受け取ることをプロデューサが要求する通知の数。可能な値: 非通知については0、リーダーのみ 1, 完全なISRについては -1。
timeout応答を待つ時間のミリ秒。
topic_datanull
トピックトピック名
datanull
パーティショントピックのパーティションid
record_setnull

Produce Request (Version: 3) => transactional_id acks timeout [topic_data] 
  transactional_id => NULLABLE_STRING
  acks => INT16
  timeout => INT32
  topic_data => topic [data] 
    topic => STRING
    data => partition record_set 
      partition => INT32
      record_set => RECORDS
フィールド 説明
transactional_idトランザクション id。あるいはプロデューサがトランザクション的ではない場合 null。
acksリクエストが完了したと見なす前にリーダーが受け取ることをプロデューサが要求する通知の数。可能な値: 非通知については0、リーダーのみ 1, 完全なISRについては -1。
timeout応答を待つ時間のミリ秒。
topic_datanull
トピックトピック名
datanull
パーティショントピックのパーティションid
record_setnull

Produce Request (Version: 4) => transactional_id acks timeout [topic_data] 
  transactional_id => NULLABLE_STRING
  acks => INT16
  timeout => INT32
  topic_data => topic [data] 
    topic => STRING
    data => partition record_set 
      partition => INT32
      record_set => RECORDS
フィールド 説明
transactional_idトランザクション id。あるいはプロデューサがトランザクション的ではない場合 null。
acksリクエストが完了したと見なす前にリーダーが受け取ることをプロデューサが要求する通知の数。可能な値: 非通知については0、リーダーのみ 1, 完全なISRについては -1。
timeout応答を待つ時間のミリ秒。
topic_datanull
トピックトピック名
datanull
パーティショントピックのパーティションid
record_setnull

Produce Request (Version: 5) => transactional_id acks timeout [topic_data] 
  transactional_id => NULLABLE_STRING
  acks => INT16
  timeout => INT32
  topic_data => topic [data] 
    topic => STRING
    data => partition record_set 
      partition => INT32
      record_set => RECORDS
フィールド 説明
transactional_idトランザクション id。あるいはプロデューサがトランザクション的ではない場合 null。
acksリクエストが完了したと見なす前にリーダーが受け取ることをプロデューサが要求する通知の数。可能な値: 非通知については0、リーダーのみ 1, 完全なISRについては -1。
timeout応答を待つ時間のミリ秒。
topic_datanull
トピックトピック名
datanull
パーティショントピックのパーティションid
record_setnull

Produce Request (Version: 6) => transactional_id acks timeout [topic_data] 
  transactional_id => NULLABLE_STRING
  acks => INT16
  timeout => INT32
  topic_data => topic [data] 
    topic => STRING
    data => partition record_set 
      partition => INT32
      record_set => RECORDS
フィールド 説明
transactional_idトランザクション id。あるいはプロデューサがトランザクション的ではない場合 null。
acksリクエストが完了したと見なす前にリーダーが受け取ることをプロデューサが要求する通知の数。可能な値: 非通知については0、リーダーのみ 1, 完全なISRについては -1。
timeout応答を待つ時間のミリ秒。
topic_datanull
トピックトピック名
datanull
パーティショントピックのパーティションid
record_setnull

Produce Request (Version: 7) => transactional_id acks timeout [topic_data] 
  transactional_id => NULLABLE_STRING
  acks => INT16
  timeout => INT32
  topic_data => topic [data] 
    topic => STRING
    data => partition record_set 
      partition => INT32
      record_set => RECORDS
フィールド 説明
transactional_idトランザクション id。あるいはプロデューサがトランザクション的ではない場合 null。
acksリクエストが完了したと見なす前にリーダーが受け取ることをプロデューサが要求する通知の数。可能な値: 非通知については0、リーダーのみ 1, 完全なISRについては -1。
timeout応答を待つ時間のミリ秒。
topic_datanull
トピックトピック名
datanull
パーティショントピックのパーティションid
record_setnull

Produce Request (Version: 8) => transactional_id acks timeout [topic_data] 
  transactional_id => NULLABLE_STRING
  acks => INT16
  timeout => INT32
  topic_data => topic [data] 
    topic => STRING
    data => partition record_set 
      partition => INT32
      record_set => RECORDS
フィールド 説明
transactional_idトランザクション id。あるいはプロデューサがトランザクション的ではない場合 null。
acksリクエストが完了したと見なす前にリーダーが受け取ることをプロデューサが要求する通知の数。可能な値: 非通知については0、リーダーのみ 1, 完全なISRについては -1。
timeout応答を待つ時間のミリ秒。
topic_datanull
トピックトピック名
datanull
パーティショントピックのパーティションid
record_setnull

Responses:

Produce Response (Version: 0) => [responses] 
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code base_offset 
      partition => INT32
      error_code => INT16
      base_offset => INT64
フィールド 説明
responsesnull
トピックトピック名
partition_responsesnull
パーティショントピックのパーティションid
error_code応答エラーコード
base_offsetnull

Produce Response (Version: 1) => [responses] throttle_time_ms 
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code base_offset 
      partition => INT32
      error_code => INT16
      base_offset => INT64
  throttle_time_ms => INT32
フィールド 説明
responsesnull
トピックトピック名
partition_responsesnull
パーティショントピックのパーティションid
error_code応答エラーコード
base_offsetnull
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間 (リクエストが何も割り当て量に違反しなかった場合は0)

Produce Response (Version: 2) => [responses] throttle_time_ms 
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code base_offset log_append_time 
      partition => INT32
      error_code => INT16
      base_offset => INT64
      log_append_time => INT64
  throttle_time_ms => INT32
フィールド 説明
responsesnull
トピックトピック名
partition_responsesnull
パーティショントピックのパーティションid
error_code応答エラーコード
base_offsetnull
log_append_timeメッセージを追加した後でブローカーによって返されるタイムスタンプ。もし CreateTime がトピックに使われる場合は、タイムスタンプは -1 でしょう。もし LogAppendTime がトピックに使われる場合は、タイムスタンプはメッセージが追加された時のブローカーのローカル時間でしょう。
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間 (リクエストが何も割り当て量に違反しなかった場合は0)

Produce Response (Version: 3) => [responses] throttle_time_ms 
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code base_offset log_append_time 
      partition => INT32
      error_code => INT16
      base_offset => INT64
      log_append_time => INT64
  throttle_time_ms => INT32
フィールド 説明
responsesnull
トピックトピック名
partition_responsesnull
パーティショントピックのパーティションid
error_code応答エラーコード
base_offsetnull
log_append_timeメッセージを追加した後でブローカーによって返されるタイムスタンプ。もし CreateTime がトピックに使われる場合は、タイムスタンプは -1 でしょう。もし LogAppendTime がトピックに使われる場合は、タイムスタンプはメッセージが追加された時のブローカーのローカル時間でしょう。
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間 (リクエストが何も割り当て量に違反しなかった場合は0)

Produce Response (Version: 4) => [responses] throttle_time_ms 
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code base_offset log_append_time 
      partition => INT32
      error_code => INT16
      base_offset => INT64
      log_append_time => INT64
  throttle_time_ms => INT32
フィールド 説明
responsesnull
トピックトピック名
partition_responsesnull
パーティショントピックのパーティションid
error_code応答エラーコード
base_offsetnull
log_append_timeメッセージを追加した後でブローカーによって返されるタイムスタンプ。もし CreateTime がトピックに使われる場合は、タイムスタンプは -1 でしょう。もし LogAppendTime がトピックに使われる場合は、タイムスタンプはメッセージが追加された時のブローカーのローカル時間でしょう。
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間 (リクエストが何も割り当て量に違反しなかった場合は0)

Produce Response (Version: 5) => [responses] throttle_time_ms 
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code base_offset log_append_time log_start_offset 
      partition => INT32
      error_code => INT16
      base_offset => INT64
      log_append_time => INT64
      log_start_offset => INT64
  throttle_time_ms => INT32
フィールド 説明
responsesnull
トピックトピック名
partition_responsesnull
パーティショントピックのパーティションid
error_code応答エラーコード
base_offsetnull
log_append_timeメッセージを追加した後でブローカーによって返されるタイムスタンプ。もし CreateTime がトピックに使われる場合は、タイムスタンプは -1 でしょう。もし LogAppendTime がトピックに使われる場合は、タイムスタンプはメッセージが追加された時のブローカーのローカル時間でしょう。
log_start_offsetこのプロデューサ応答が作成された時のログの開始オフセット
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間 (リクエストが何も割り当て量に違反しなかった場合は0)

Produce Response (Version: 6) => [responses] throttle_time_ms 
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code base_offset log_append_time log_start_offset 
      partition => INT32
      error_code => INT16
      base_offset => INT64
      log_append_time => INT64
      log_start_offset => INT64
  throttle_time_ms => INT32
フィールド 説明
responsesnull
トピックトピック名
partition_responsesnull
パーティショントピックのパーティションid
error_code応答エラーコード
base_offsetnull
log_append_timeメッセージを追加した後でブローカーによって返されるタイムスタンプ。もし CreateTime がトピックに使われる場合は、タイムスタンプは -1 でしょう。もし LogAppendTime がトピックに使われる場合は、タイムスタンプはメッセージが追加された時のブローカーのローカル時間でしょう。
log_start_offsetこのプロデューサ応答が作成された時のログの開始オフセット
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間 (リクエストが何も割り当て量に違反しなかった場合は0)

Produce Response (Version: 7) => [responses] throttle_time_ms 
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code base_offset log_append_time log_start_offset 
      partition => INT32
      error_code => INT16
      base_offset => INT64
      log_append_time => INT64
      log_start_offset => INT64
  throttle_time_ms => INT32
フィールド 説明
responsesnull
トピックトピック名
partition_responsesnull
パーティショントピックのパーティションid
error_code応答エラーコード
base_offsetnull
log_append_timeメッセージを追加した後でブローカーによって返されるタイムスタンプ。もし CreateTime がトピックに使われる場合は、タイムスタンプは -1 でしょう。もし LogAppendTime がトピックに使われる場合は、タイムスタンプはメッセージが追加された時のブローカーのローカル時間でしょう。
log_start_offsetこのプロデューサ応答が作成された時のログの開始オフセット
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間 (リクエストが何も割り当て量に違反しなかった場合は0)

Produce Response (Version: 8) => [responses] throttle_time_ms 
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code base_offset log_append_time log_start_offset [record_errors] error_message 
      partition => INT32
      error_code => INT16
      base_offset => INT64
      log_append_time => INT64
      log_start_offset => INT64
      record_errors => batch_index batch_index_error_message 
        batch_index => INT32
        batch_index_error_message => NULLABLE_STRING
      error_message => NULLABLE_STRING
  throttle_time_ms => INT32
フィールド 説明
responsesnull
トピックトピック名
partition_responsesnull
パーティショントピックのパーティションid
error_code応答エラーコード
base_offsetnull
log_append_timeメッセージを追加した後でブローカーによって返されるタイムスタンプ。もし CreateTime がトピックに使われる場合は、タイムスタンプは -1 でしょう。もし LogAppendTime がトピックに使われる場合は、タイムスタンプはメッセージが追加された時のブローカーのローカル時間でしょう。
log_start_offsetこのプロデューサ応答が作成された時のログの開始オフセット
record_errorsバッチが削除される原因となったレコードのバッチインデックス
batch_indexバッチが削除される原因となったレコードのバッチインデックス
batch_index_error_messageバッチが削除される原因となったレコードのエラーメッセージ
error_messageバッチが削除される原因となったレコードの一般的な根本原因を要約したグローバルエラーメッセージ
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間 (リクエストが何も割り当て量に違反しなかった場合は0)

Fetch API (Key: 1):
Requests:

Fetch Request (Version: 0) => replica_id max_wait_ms min_bytes [topics] 
  replica_id => INT32
  max_wait_ms => INT32
  min_bytes => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => partition fetch_offset partition_max_bytes 
      partition => INT32
      fetch_offset => INT64
      partition_max_bytes => INT32
フィールド 説明
replica_idThe broker ID of the follower, of -1 if this request is from a consumer.
max_wait_msThe maximum time in milliseconds to wait for the response.
min_bytesThe minimum bytes to accumulate in the response.
topicsThe topics to fetch.
トピックThe name of the topic to fetch.
partitionsThe partitions to fetch.
パーティションパーティションのインデックス。
fetch_offsetThe message offset.
partition_max_bytesThe maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored.

Fetch Request (Version: 1) => replica_id max_wait_ms min_bytes [topics] 
  replica_id => INT32
  max_wait_ms => INT32
  min_bytes => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => partition fetch_offset partition_max_bytes 
      partition => INT32
      fetch_offset => INT64
      partition_max_bytes => INT32
フィールド 説明
replica_idThe broker ID of the follower, of -1 if this request is from a consumer.
max_wait_msThe maximum time in milliseconds to wait for the response.
min_bytesThe minimum bytes to accumulate in the response.
topicsThe topics to fetch.
トピックThe name of the topic to fetch.
partitionsThe partitions to fetch.
パーティションパーティションのインデックス。
fetch_offsetThe message offset.
partition_max_bytesThe maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored.

Fetch Request (Version: 2) => replica_id max_wait_ms min_bytes [topics] 
  replica_id => INT32
  max_wait_ms => INT32
  min_bytes => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => partition fetch_offset partition_max_bytes 
      partition => INT32
      fetch_offset => INT64
      partition_max_bytes => INT32
フィールド 説明
replica_idThe broker ID of the follower, of -1 if this request is from a consumer.
max_wait_msThe maximum time in milliseconds to wait for the response.
min_bytesThe minimum bytes to accumulate in the response.
topicsThe topics to fetch.
トピックThe name of the topic to fetch.
partitionsThe partitions to fetch.
パーティションパーティションのインデックス。
fetch_offsetThe message offset.
partition_max_bytesThe maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored.

Fetch Request (Version: 3) => replica_id max_wait_ms min_bytes max_bytes [topics] 
  replica_id => INT32
  max_wait_ms => INT32
  min_bytes => INT32
  max_bytes => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => partition fetch_offset partition_max_bytes 
      partition => INT32
      fetch_offset => INT64
      partition_max_bytes => INT32
フィールド 説明
replica_idThe broker ID of the follower, of -1 if this request is from a consumer.
max_wait_msThe maximum time in milliseconds to wait for the response.
min_bytesThe minimum bytes to accumulate in the response.
max_bytesThe maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored.
topicsThe topics to fetch.
トピックThe name of the topic to fetch.
partitionsThe partitions to fetch.
パーティションパーティションのインデックス。
fetch_offsetThe message offset.
partition_max_bytesThe maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored.

Fetch Request (Version: 4) => replica_id max_wait_ms min_bytes max_bytes isolation_level [topics] 
  replica_id => INT32
  max_wait_ms => INT32
  min_bytes => INT32
  max_bytes => INT32
  isolation_level => INT8
  topics => topic [partitions] 
    topic => STRING
    partitions => partition fetch_offset partition_max_bytes 
      partition => INT32
      fetch_offset => INT64
      partition_max_bytes => INT32
フィールド 説明
replica_idThe broker ID of the follower, of -1 if this request is from a consumer.
max_wait_msThe maximum time in milliseconds to wait for the response.
min_bytesThe minimum bytes to accumulate in the response.
max_bytesThe maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored.
isolation_levelこの設定はトランザクションのレコードの可視性を制御します。READ_UNCOMMITTED (isolation_level = 0) の使用により全てのレコードが見えるようになります。READ_COMMITTED (isolation_level = 1) を使うと、非トランザクションとCOMMITTEDトランザクションレコードが見えるようになります。もっと具体的には、READ_COMMITTED は現在のLSO(最後の安定したオフセット)よりも小さなオフセットから全てのデータを返し、結果の中の中断されたトランザクションのリストを含めるようにします。これによりコンシューマはABORTEDのトランザクションレコードを破棄することができます。
topicsThe topics to fetch.
トピックThe name of the topic to fetch.
partitionsThe partitions to fetch.
パーティションパーティションのインデックス。
fetch_offsetThe message offset.
partition_max_bytesThe maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored.

Fetch Request (Version: 5) => replica_id max_wait_ms min_bytes max_bytes isolation_level [topics] 
  replica_id => INT32
  max_wait_ms => INT32
  min_bytes => INT32
  max_bytes => INT32
  isolation_level => INT8
  topics => topic [partitions] 
    topic => STRING
    partitions => partition fetch_offset log_start_offset partition_max_bytes 
      partition => INT32
      fetch_offset => INT64
      log_start_offset => INT64
      partition_max_bytes => INT32
フィールド 説明
replica_idThe broker ID of the follower, of -1 if this request is from a consumer.
max_wait_msThe maximum time in milliseconds to wait for the response.
min_bytesThe minimum bytes to accumulate in the response.
max_bytesThe maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored.
isolation_levelこの設定はトランザクションのレコードの可視性を制御します。READ_UNCOMMITTED (isolation_level = 0) の使用により全てのレコードが見えるようになります。READ_COMMITTED (isolation_level = 1) を使うと、非トランザクションとCOMMITTEDトランザクションレコードが見えるようになります。もっと具体的には、READ_COMMITTED は現在のLSO(最後の安定したオフセット)よりも小さなオフセットから全てのデータを返し、結果の中の中断されたトランザクションのリストを含めるようにします。これによりコンシューマはABORTEDのトランザクションレコードを破棄することができます。
topicsThe topics to fetch.
トピックThe name of the topic to fetch.
partitionsThe partitions to fetch.
パーティションパーティションのインデックス。
fetch_offsetThe message offset.
log_start_offsetThe earliest available offset of the follower replica. The field is only used when the request is sent by the follower.
partition_max_bytesThe maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored.

Fetch Request (Version: 6) => replica_id max_wait_ms min_bytes max_bytes isolation_level [topics] 
  replica_id => INT32
  max_wait_ms => INT32
  min_bytes => INT32
  max_bytes => INT32
  isolation_level => INT8
  topics => topic [partitions] 
    topic => STRING
    partitions => partition fetch_offset log_start_offset partition_max_bytes 
      partition => INT32
      fetch_offset => INT64
      log_start_offset => INT64
      partition_max_bytes => INT32
フィールド 説明
replica_idThe broker ID of the follower, of -1 if this request is from a consumer.
max_wait_msThe maximum time in milliseconds to wait for the response.
min_bytesThe minimum bytes to accumulate in the response.
max_bytesThe maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored.
isolation_levelこの設定はトランザクションのレコードの可視性を制御します。READ_UNCOMMITTED (isolation_level = 0) の使用により全てのレコードが見えるようになります。READ_COMMITTED (isolation_level = 1) を使うと、非トランザクションとCOMMITTEDトランザクションレコードが見えるようになります。もっと具体的には、READ_COMMITTED は現在のLSO(最後の安定したオフセット)よりも小さなオフセットから全てのデータを返し、結果の中の中断されたトランザクションのリストを含めるようにします。これによりコンシューマはABORTEDのトランザクションレコードを破棄することができます。
topicsThe topics to fetch.
トピックThe name of the topic to fetch.
partitionsThe partitions to fetch.
パーティションパーティションのインデックス。
fetch_offsetThe message offset.
log_start_offsetThe earliest available offset of the follower replica. The field is only used when the request is sent by the follower.
partition_max_bytesThe maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored.

Fetch Request (Version: 7) => replica_id max_wait_ms min_bytes max_bytes isolation_level session_id session_epoch [topics] [forgotten_topics_data] 
  replica_id => INT32
  max_wait_ms => INT32
  min_bytes => INT32
  max_bytes => INT32
  isolation_level => INT8
  session_id => INT32
  session_epoch => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => partition fetch_offset log_start_offset partition_max_bytes 
      partition => INT32
      fetch_offset => INT64
      log_start_offset => INT64
      partition_max_bytes => INT32
  forgotten_topics_data => topic [partitions] 
    topic => STRING
    partitions => INT32
フィールド 説明
replica_idThe broker ID of the follower, of -1 if this request is from a consumer.
max_wait_msThe maximum time in milliseconds to wait for the response.
min_bytesThe minimum bytes to accumulate in the response.
max_bytesThe maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored.
isolation_levelこの設定はトランザクションのレコードの可視性を制御します。READ_UNCOMMITTED (isolation_level = 0) の使用により全てのレコードが見えるようになります。READ_COMMITTED (isolation_level = 1) を使うと、非トランザクションとCOMMITTEDトランザクションレコードが見えるようになります。もっと具体的には、READ_COMMITTED は現在のLSO(最後の安定したオフセット)よりも小さなオフセットから全てのデータを返し、結果の中の中断されたトランザクションのリストを含めるようにします。これによりコンシューマはABORTEDのトランザクションレコードを破棄することができます。
session_idThe fetch session ID.
session_epochThe fetch session epoch, which is used for ordering requests in a session.
topicsThe topics to fetch.
トピックThe name of the topic to fetch.
partitionsThe partitions to fetch.
パーティションパーティションのインデックス。
fetch_offsetThe message offset.
log_start_offsetThe earliest available offset of the follower replica. The field is only used when the request is sent by the follower.
partition_max_bytesThe maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored.
forgotten_topics_dataIn an incremental fetch request, the partitions to remove.
トピックThe partition name.
partitionsThe partitions indexes to forget.

Fetch Request (Version: 8) => replica_id max_wait_ms min_bytes max_bytes isolation_level session_id session_epoch [topics] [forgotten_topics_data] 
  replica_id => INT32
  max_wait_ms => INT32
  min_bytes => INT32
  max_bytes => INT32
  isolation_level => INT8
  session_id => INT32
  session_epoch => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => partition fetch_offset log_start_offset partition_max_bytes 
      partition => INT32
      fetch_offset => INT64
      log_start_offset => INT64
      partition_max_bytes => INT32
  forgotten_topics_data => topic [partitions] 
    topic => STRING
    partitions => INT32
フィールド 説明
replica_idThe broker ID of the follower, of -1 if this request is from a consumer.
max_wait_msThe maximum time in milliseconds to wait for the response.
min_bytesThe minimum bytes to accumulate in the response.
max_bytesThe maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored.
isolation_levelこの設定はトランザクションのレコードの可視性を制御します。READ_UNCOMMITTED (isolation_level = 0) の使用により全てのレコードが見えるようになります。READ_COMMITTED (isolation_level = 1) を使うと、非トランザクションとCOMMITTEDトランザクションレコードが見えるようになります。もっと具体的には、READ_COMMITTED は現在のLSO(最後の安定したオフセット)よりも小さなオフセットから全てのデータを返し、結果の中の中断されたトランザクションのリストを含めるようにします。これによりコンシューマはABORTEDのトランザクションレコードを破棄することができます。
session_idThe fetch session ID.
session_epochThe fetch session epoch, which is used for ordering requests in a session.
topicsThe topics to fetch.
トピックThe name of the topic to fetch.
partitionsThe partitions to fetch.
パーティションパーティションのインデックス。
fetch_offsetThe message offset.
log_start_offsetThe earliest available offset of the follower replica. The field is only used when the request is sent by the follower.
partition_max_bytesThe maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored.
forgotten_topics_dataIn an incremental fetch request, the partitions to remove.
トピックThe partition name.
partitionsThe partitions indexes to forget.

Fetch Request (Version: 9) => replica_id max_wait_ms min_bytes max_bytes isolation_level session_id session_epoch [topics] [forgotten_topics_data] 
  replica_id => INT32
  max_wait_ms => INT32
  min_bytes => INT32
  max_bytes => INT32
  isolation_level => INT8
  session_id => INT32
  session_epoch => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => partition current_leader_epoch fetch_offset log_start_offset partition_max_bytes 
      partition => INT32
      current_leader_epoch => INT32
      fetch_offset => INT64
      log_start_offset => INT64
      partition_max_bytes => INT32
  forgotten_topics_data => topic [partitions] 
    topic => STRING
    partitions => INT32
フィールド 説明
replica_idThe broker ID of the follower, of -1 if this request is from a consumer.
max_wait_msThe maximum time in milliseconds to wait for the response.
min_bytesThe minimum bytes to accumulate in the response.
max_bytesThe maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored.
isolation_levelこの設定はトランザクションのレコードの可視性を制御します。READ_UNCOMMITTED (isolation_level = 0) の使用により全てのレコードが見えるようになります。READ_COMMITTED (isolation_level = 1) を使うと、非トランザクションとCOMMITTEDトランザクションレコードが見えるようになります。もっと具体的には、READ_COMMITTED は現在のLSO(最後の安定したオフセット)よりも小さなオフセットから全てのデータを返し、結果の中の中断されたトランザクションのリストを含めるようにします。これによりコンシューマはABORTEDのトランザクションレコードを破棄することができます。
session_idThe fetch session ID.
session_epochThe fetch session epoch, which is used for ordering requests in a session.
topicsThe topics to fetch.
トピックThe name of the topic to fetch.
partitionsThe partitions to fetch.
パーティションパーティションのインデックス。
current_leader_epochThe current leader epoch of the partition.
fetch_offsetThe message offset.
log_start_offsetThe earliest available offset of the follower replica. The field is only used when the request is sent by the follower.
partition_max_bytesThe maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored.
forgotten_topics_dataIn an incremental fetch request, the partitions to remove.
トピックThe partition name.
partitionsThe partitions indexes to forget.

Fetch Request (Version: 10) => replica_id max_wait_ms min_bytes max_bytes isolation_level session_id session_epoch [topics] [forgotten_topics_data] 
  replica_id => INT32
  max_wait_ms => INT32
  min_bytes => INT32
  max_bytes => INT32
  isolation_level => INT8
  session_id => INT32
  session_epoch => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => partition current_leader_epoch fetch_offset log_start_offset partition_max_bytes 
      partition => INT32
      current_leader_epoch => INT32
      fetch_offset => INT64
      log_start_offset => INT64
      partition_max_bytes => INT32
  forgotten_topics_data => topic [partitions] 
    topic => STRING
    partitions => INT32
フィールド 説明
replica_idThe broker ID of the follower, of -1 if this request is from a consumer.
max_wait_msThe maximum time in milliseconds to wait for the response.
min_bytesThe minimum bytes to accumulate in the response.
max_bytesThe maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored.
isolation_levelこの設定はトランザクションのレコードの可視性を制御します。READ_UNCOMMITTED (isolation_level = 0) の使用により全てのレコードが見えるようになります。READ_COMMITTED (isolation_level = 1) を使うと、非トランザクションとCOMMITTEDトランザクションレコードが見えるようになります。もっと具体的には、READ_COMMITTED は現在のLSO(最後の安定したオフセット)よりも小さなオフセットから全てのデータを返し、結果の中の中断されたトランザクションのリストを含めるようにします。これによりコンシューマはABORTEDのトランザクションレコードを破棄することができます。
session_idThe fetch session ID.
session_epochThe fetch session epoch, which is used for ordering requests in a session.
topicsThe topics to fetch.
トピックThe name of the topic to fetch.
partitionsThe partitions to fetch.
パーティションパーティションのインデックス。
current_leader_epochThe current leader epoch of the partition.
fetch_offsetThe message offset.
log_start_offsetThe earliest available offset of the follower replica. The field is only used when the request is sent by the follower.
partition_max_bytesThe maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored.
forgotten_topics_dataIn an incremental fetch request, the partitions to remove.
トピックThe partition name.
partitionsThe partitions indexes to forget.

Fetch Request (Version: 11) => replica_id max_wait_ms min_bytes max_bytes isolation_level session_id session_epoch [topics] [forgotten_topics_data] rack_id 
  replica_id => INT32
  max_wait_ms => INT32
  min_bytes => INT32
  max_bytes => INT32
  isolation_level => INT8
  session_id => INT32
  session_epoch => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => partition current_leader_epoch fetch_offset log_start_offset partition_max_bytes 
      partition => INT32
      current_leader_epoch => INT32
      fetch_offset => INT64
      log_start_offset => INT64
      partition_max_bytes => INT32
  forgotten_topics_data => topic [partitions] 
    topic => STRING
    partitions => INT32
  rack_id => STRING
フィールド 説明
replica_idThe broker ID of the follower, of -1 if this request is from a consumer.
max_wait_msThe maximum time in milliseconds to wait for the response.
min_bytesThe minimum bytes to accumulate in the response.
max_bytesThe maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored.
isolation_levelこの設定はトランザクションのレコードの可視性を制御します。READ_UNCOMMITTED (isolation_level = 0) の使用により全てのレコードが見えるようになります。READ_COMMITTED (isolation_level = 1) を使うと、非トランザクションとCOMMITTEDトランザクションレコードが見えるようになります。もっと具体的には、READ_COMMITTED は現在のLSO(最後の安定したオフセット)よりも小さなオフセットから全てのデータを返し、結果の中の中断されたトランザクションのリストを含めるようにします。これによりコンシューマはABORTEDのトランザクションレコードを破棄することができます。
session_idThe fetch session ID.
session_epochThe fetch session epoch, which is used for ordering requests in a session.
topicsThe topics to fetch.
トピックThe name of the topic to fetch.
partitionsThe partitions to fetch.
パーティションパーティションのインデックス。
current_leader_epochThe current leader epoch of the partition.
fetch_offsetThe message offset.
log_start_offsetThe earliest available offset of the follower replica. The field is only used when the request is sent by the follower.
partition_max_bytesThe maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored.
forgotten_topics_dataIn an incremental fetch request, the partitions to remove.
トピックThe partition name.
partitionsThe partitions indexes to forget.
rack_idRack ID of the consumer making this request

Fetch Request (Version: 12) => replica_id max_wait_ms min_bytes max_bytes isolation_level session_id session_epoch [topics] [forgotten_topics_data] rack_id TAG_BUFFER 
  replica_id => INT32
  max_wait_ms => INT32
  min_bytes => INT32
  max_bytes => INT32
  isolation_level => INT8
  session_id => INT32
  session_epoch => INT32
  topics => topic [partitions] TAG_BUFFER 
    topic => COMPACT_STRING
    partitions => partition current_leader_epoch fetch_offset last_fetched_epoch log_start_offset partition_max_bytes TAG_BUFFER 
      partition => INT32
      current_leader_epoch => INT32
      fetch_offset => INT64
      last_fetched_epoch => INT32
      log_start_offset => INT64
      partition_max_bytes => INT32
  forgotten_topics_data => topic [partitions] TAG_BUFFER 
    topic => COMPACT_STRING
    partitions => INT32
  rack_id => COMPACT_STRING
フィールド 説明
replica_idThe broker ID of the follower, of -1 if this request is from a consumer.
max_wait_msThe maximum time in milliseconds to wait for the response.
min_bytesThe minimum bytes to accumulate in the response.
max_bytesThe maximum bytes to fetch. See KIP-74 for cases where this limit may not be honored.
isolation_levelこの設定はトランザクションのレコードの可視性を制御します。READ_UNCOMMITTED (isolation_level = 0) の使用により全てのレコードが見えるようになります。READ_COMMITTED (isolation_level = 1) を使うと、非トランザクションとCOMMITTEDトランザクションレコードが見えるようになります。もっと具体的には、READ_COMMITTED は現在のLSO(最後の安定したオフセット)よりも小さなオフセットから全てのデータを返し、結果の中の中断されたトランザクションのリストを含めるようにします。これによりコンシューマはABORTEDのトランザクションレコードを破棄することができます。
session_idThe fetch session ID.
session_epochThe fetch session epoch, which is used for ordering requests in a session.
topicsThe topics to fetch.
トピックThe name of the topic to fetch.
partitionsThe partitions to fetch.
パーティションパーティションのインデックス。
current_leader_epochThe current leader epoch of the partition.
fetch_offsetThe message offset.
last_fetched_epochThe epoch of the last fetched record or -1 if there is none
log_start_offsetThe earliest available offset of the follower replica. The field is only used when the request is sent by the follower.
partition_max_bytesThe maximum bytes to fetch from this partition. See KIP-74 for cases where this limit may not be honored.
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド
forgotten_topics_dataIn an incremental fetch request, the partitions to remove.
トピックThe partition name.
partitionsThe partitions indexes to forget.
_tagged_fieldsタグ付きのフィールド
rack_idRack ID of the consumer making this request
_tagged_fieldsタグ付きのフィールド

Responses:

Fetch Response (Version: 0) => [responses] 
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code high_watermark record_set 
      partition => INT32
      error_code => INT16
      high_watermark => INT64
      record_set => RECORDS
フィールド 説明
responsesThe response topics.
トピックトピック名。
partition_responsesThe topic partitions.
パーティションパーティションのインデックス。
error_codeThe error code, or 0 if there was no fetch error.
high_watermarkThe current high water mark.
record_setThe record data.

Fetch Response (Version: 1) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code high_watermark record_set 
      partition => INT32
      error_code => INT16
      high_watermark => INT64
      record_set => RECORDS
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
responsesThe response topics.
トピックトピック名。
partition_responsesThe topic partitions.
パーティションパーティションのインデックス。
error_codeThe error code, or 0 if there was no fetch error.
high_watermarkThe current high water mark.
record_setThe record data.

Fetch Response (Version: 2) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code high_watermark record_set 
      partition => INT32
      error_code => INT16
      high_watermark => INT64
      record_set => RECORDS
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
responsesThe response topics.
トピックトピック名。
partition_responsesThe topic partitions.
パーティションパーティションのインデックス。
error_codeThe error code, or 0 if there was no fetch error.
high_watermarkThe current high water mark.
record_setThe record data.

Fetch Response (Version: 3) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code high_watermark record_set 
      partition => INT32
      error_code => INT16
      high_watermark => INT64
      record_set => RECORDS
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
responsesThe response topics.
トピックトピック名。
partition_responsesThe topic partitions.
パーティションパーティションのインデックス。
error_codeThe error code, or 0 if there was no fetch error.
high_watermarkThe current high water mark.
record_setThe record data.

Fetch Response (Version: 4) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code high_watermark last_stable_offset [aborted_transactions] record_set 
      partition => INT32
      error_code => INT16
      high_watermark => INT64
      last_stable_offset => INT64
      aborted_transactions => producer_id first_offset 
        producer_id => INT64
        first_offset => INT64
      record_set => RECORDS
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
responsesThe response topics.
トピックトピック名。
partition_responsesThe topic partitions.
パーティションパーティションのインデックス。
error_codeThe error code, or 0 if there was no fetch error.
high_watermarkThe current high water mark.
last_stable_offsetパーティションの最後の安定したオフセット(あるいはLSO)。これは最後のオフセットであり、このオフセットの前の全てのトランザクション レコードの状態が決定されています (ABORTED あるいは COMMITTED)
aborted_transactionsThe aborted transactions.
producer_idThe producer id associated with the aborted transaction.
first_offsetThe first offset in the aborted transaction.
record_setThe record data.

Fetch Response (Version: 5) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code high_watermark last_stable_offset log_start_offset [aborted_transactions] record_set 
      partition => INT32
      error_code => INT16
      high_watermark => INT64
      last_stable_offset => INT64
      log_start_offset => INT64
      aborted_transactions => producer_id first_offset 
        producer_id => INT64
        first_offset => INT64
      record_set => RECORDS
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
responsesThe response topics.
トピックトピック名。
partition_responsesThe topic partitions.
パーティションパーティションのインデックス。
error_codeThe error code, or 0 if there was no fetch error.
high_watermarkThe current high water mark.
last_stable_offsetパーティションの最後の安定したオフセット(あるいはLSO)。これは最後のオフセットであり、このオフセットの前の全てのトランザクション レコードの状態が決定されています (ABORTED あるいは COMMITTED)
log_start_offsetThe current log start offset.
aborted_transactionsThe aborted transactions.
producer_idThe producer id associated with the aborted transaction.
first_offsetThe first offset in the aborted transaction.
record_setThe record data.

Fetch Response (Version: 6) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code high_watermark last_stable_offset log_start_offset [aborted_transactions] record_set 
      partition => INT32
      error_code => INT16
      high_watermark => INT64
      last_stable_offset => INT64
      log_start_offset => INT64
      aborted_transactions => producer_id first_offset 
        producer_id => INT64
        first_offset => INT64
      record_set => RECORDS
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
responsesThe response topics.
トピックトピック名。
partition_responsesThe topic partitions.
パーティションパーティションのインデックス。
error_codeThe error code, or 0 if there was no fetch error.
high_watermarkThe current high water mark.
last_stable_offsetパーティションの最後の安定したオフセット(あるいはLSO)。これは最後のオフセットであり、このオフセットの前の全てのトランザクション レコードの状態が決定されています (ABORTED あるいは COMMITTED)
log_start_offsetThe current log start offset.
aborted_transactionsThe aborted transactions.
producer_idThe producer id associated with the aborted transaction.
first_offsetThe first offset in the aborted transaction.
record_setThe record data.

Fetch Response (Version: 7) => throttle_time_ms error_code session_id [responses] 
  throttle_time_ms => INT32
  error_code => INT16
  session_id => INT32
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code high_watermark last_stable_offset log_start_offset [aborted_transactions] record_set 
      partition => INT32
      error_code => INT16
      high_watermark => INT64
      last_stable_offset => INT64
      log_start_offset => INT64
      aborted_transactions => producer_id first_offset 
        producer_id => INT64
        first_offset => INT64
      record_set => RECORDS
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeトップレベルの応答エラーコード。
session_idThe fetch session ID, or 0 if this is not part of a fetch session.
responsesThe response topics.
トピックトピック名。
partition_responsesThe topic partitions.
パーティションパーティションのインデックス。
error_codeThe error code, or 0 if there was no fetch error.
high_watermarkThe current high water mark.
last_stable_offsetパーティションの最後の安定したオフセット(あるいはLSO)。これは最後のオフセットであり、このオフセットの前の全てのトランザクション レコードの状態が決定されています (ABORTED あるいは COMMITTED)
log_start_offsetThe current log start offset.
aborted_transactionsThe aborted transactions.
producer_idThe producer id associated with the aborted transaction.
first_offsetThe first offset in the aborted transaction.
record_setThe record data.

Fetch Response (Version: 8) => throttle_time_ms error_code session_id [responses] 
  throttle_time_ms => INT32
  error_code => INT16
  session_id => INT32
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code high_watermark last_stable_offset log_start_offset [aborted_transactions] record_set 
      partition => INT32
      error_code => INT16
      high_watermark => INT64
      last_stable_offset => INT64
      log_start_offset => INT64
      aborted_transactions => producer_id first_offset 
        producer_id => INT64
        first_offset => INT64
      record_set => RECORDS
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeトップレベルの応答エラーコード。
session_idThe fetch session ID, or 0 if this is not part of a fetch session.
responsesThe response topics.
トピックトピック名。
partition_responsesThe topic partitions.
パーティションパーティションのインデックス。
error_codeThe error code, or 0 if there was no fetch error.
high_watermarkThe current high water mark.
last_stable_offsetパーティションの最後の安定したオフセット(あるいはLSO)。これは最後のオフセットであり、このオフセットの前の全てのトランザクション レコードの状態が決定されています (ABORTED あるいは COMMITTED)
log_start_offsetThe current log start offset.
aborted_transactionsThe aborted transactions.
producer_idThe producer id associated with the aborted transaction.
first_offsetThe first offset in the aborted transaction.
record_setThe record data.

Fetch Response (Version: 9) => throttle_time_ms error_code session_id [responses] 
  throttle_time_ms => INT32
  error_code => INT16
  session_id => INT32
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code high_watermark last_stable_offset log_start_offset [aborted_transactions] record_set 
      partition => INT32
      error_code => INT16
      high_watermark => INT64
      last_stable_offset => INT64
      log_start_offset => INT64
      aborted_transactions => producer_id first_offset 
        producer_id => INT64
        first_offset => INT64
      record_set => RECORDS
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeトップレベルの応答エラーコード。
session_idThe fetch session ID, or 0 if this is not part of a fetch session.
responsesThe response topics.
トピックトピック名。
partition_responsesThe topic partitions.
パーティションパーティションのインデックス。
error_codeThe error code, or 0 if there was no fetch error.
high_watermarkThe current high water mark.
last_stable_offsetパーティションの最後の安定したオフセット(あるいはLSO)。これは最後のオフセットであり、このオフセットの前の全てのトランザクション レコードの状態が決定されています (ABORTED あるいは COMMITTED)
log_start_offsetThe current log start offset.
aborted_transactionsThe aborted transactions.
producer_idThe producer id associated with the aborted transaction.
first_offsetThe first offset in the aborted transaction.
record_setThe record data.

Fetch Response (Version: 10) => throttle_time_ms error_code session_id [responses] 
  throttle_time_ms => INT32
  error_code => INT16
  session_id => INT32
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code high_watermark last_stable_offset log_start_offset [aborted_transactions] record_set 
      partition => INT32
      error_code => INT16
      high_watermark => INT64
      last_stable_offset => INT64
      log_start_offset => INT64
      aborted_transactions => producer_id first_offset 
        producer_id => INT64
        first_offset => INT64
      record_set => RECORDS
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeトップレベルの応答エラーコード。
session_idThe fetch session ID, or 0 if this is not part of a fetch session.
responsesThe response topics.
トピックトピック名。
partition_responsesThe topic partitions.
パーティションパーティションのインデックス。
error_codeThe error code, or 0 if there was no fetch error.
high_watermarkThe current high water mark.
last_stable_offsetパーティションの最後の安定したオフセット(あるいはLSO)。これは最後のオフセットであり、このオフセットの前の全てのトランザクション レコードの状態が決定されています (ABORTED あるいは COMMITTED)
log_start_offsetThe current log start offset.
aborted_transactionsThe aborted transactions.
producer_idThe producer id associated with the aborted transaction.
first_offsetThe first offset in the aborted transaction.
record_setThe record data.

Fetch Response (Version: 11) => throttle_time_ms error_code session_id [responses] 
  throttle_time_ms => INT32
  error_code => INT16
  session_id => INT32
  responses => topic [partition_responses] 
    topic => STRING
    partition_responses => partition error_code high_watermark last_stable_offset log_start_offset [aborted_transactions] preferred_read_replica record_set 
      partition => INT32
      error_code => INT16
      high_watermark => INT64
      last_stable_offset => INT64
      log_start_offset => INT64
      aborted_transactions => producer_id first_offset 
        producer_id => INT64
        first_offset => INT64
      preferred_read_replica => INT32
      record_set => RECORDS
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeトップレベルの応答エラーコード。
session_idThe fetch session ID, or 0 if this is not part of a fetch session.
responsesThe response topics.
トピックトピック名。
partition_responsesThe topic partitions.
パーティションパーティションのインデックス。
error_codeThe error code, or 0 if there was no fetch error.
high_watermarkThe current high water mark.
last_stable_offsetパーティションの最後の安定したオフセット(あるいはLSO)。これは最後のオフセットであり、このオフセットの前の全てのトランザクション レコードの状態が決定されています (ABORTED あるいは COMMITTED)
log_start_offsetThe current log start offset.
aborted_transactionsThe aborted transactions.
producer_idThe producer id associated with the aborted transaction.
first_offsetThe first offset in the aborted transaction.
preferred_read_replicaThe preferred read replica for the consumer to use on its next fetch request
record_setThe record data.

Fetch Response (Version: 12) => throttle_time_ms error_code session_id [responses] TAG_BUFFER 
  throttle_time_ms => INT32
  error_code => INT16
  session_id => INT32
  responses => topic [partition_responses] TAG_BUFFER 
    topic => COMPACT_STRING
    partition_responses => partition error_code high_watermark last_stable_offset log_start_offset [aborted_transactions] preferred_read_replica record_set TAG_BUFFER 
      partition => INT32
      error_code => INT16
      high_watermark => INT64
      last_stable_offset => INT64
      log_start_offset => INT64
      aborted_transactions => producer_id first_offset TAG_BUFFER 
        producer_id => INT64
        first_offset => INT64
      preferred_read_replica => INT32
      record_set => COMPACT_RECORDS
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeトップレベルの応答エラーコード。
session_idThe fetch session ID, or 0 if this is not part of a fetch session.
responsesThe response topics.
トピックトピック名。
partition_responsesThe topic partitions.
パーティションパーティションのインデックス。
error_codeThe error code, or 0 if there was no fetch error.
high_watermarkThe current high water mark.
last_stable_offsetパーティションの最後の安定したオフセット(あるいはLSO)。これは最後のオフセットであり、このオフセットの前の全てのトランザクション レコードの状態が決定されています (ABORTED あるいは COMMITTED)
log_start_offsetThe current log start offset.
aborted_transactionsThe aborted transactions.
producer_idThe producer id associated with the aborted transaction.
first_offsetThe first offset in the aborted transaction.
_tagged_fieldsタグ付きのフィールド
preferred_read_replicaThe preferred read replica for the consumer to use on its next fetch request
record_setThe record data.
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

ListOffsets API (Key: 2):
Requests:

ListOffsets Request (Version: 0) => replica_id [topics] 
  replica_id => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index timestamp max_num_offsets 
      partition_index => INT32
      timestamp => INT64
      max_num_offsets => INT32
フィールド 説明
replica_idThe broker ID of the requestor, or -1 if this request is being made by a normal consumer.
topicsEach topic in the request.
名前トピック名。
partitionsEach partition in the request.
partition_indexパーティションのインデックス。
timestampThe current timestamp.
max_num_offsetsThe maximum number of offsets to report.

ListOffsets Request (Version: 1) => replica_id [topics] 
  replica_id => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index timestamp 
      partition_index => INT32
      timestamp => INT64
フィールド 説明
replica_idThe broker ID of the requestor, or -1 if this request is being made by a normal consumer.
topicsEach topic in the request.
名前トピック名。
partitionsEach partition in the request.
partition_indexパーティションのインデックス。
timestampThe current timestamp.

ListOffsets Request (Version: 2) => replica_id isolation_level [topics] 
  replica_id => INT32
  isolation_level => INT8
  topics => name [partitions] 
    name => STRING
    partitions => partition_index timestamp 
      partition_index => INT32
      timestamp => INT64
フィールド 説明
replica_idThe broker ID of the requestor, or -1 if this request is being made by a normal consumer.
isolation_levelこの設定はトランザクションのレコードの可視性を制御します。READ_UNCOMMITTED (isolation_level = 0) の使用により全てのレコードが見えるようになります。READ_COMMITTED (isolation_level = 1) を使うと、非トランザクションとCOMMITTEDトランザクションレコードが見えるようになります。もっと具体的には、READ_COMMITTED は現在のLSO(最後の安定したオフセット)よりも小さなオフセットから全てのデータを返し、結果の中の中断されたトランザクションのリストを含めるようにします。これによりコンシューマはABORTEDのトランザクションレコードを破棄することができます。
topicsEach topic in the request.
名前トピック名。
partitionsEach partition in the request.
partition_indexパーティションのインデックス。
timestampThe current timestamp.

ListOffsets Request (Version: 3) => replica_id isolation_level [topics] 
  replica_id => INT32
  isolation_level => INT8
  topics => name [partitions] 
    name => STRING
    partitions => partition_index timestamp 
      partition_index => INT32
      timestamp => INT64
フィールド 説明
replica_idThe broker ID of the requestor, or -1 if this request is being made by a normal consumer.
isolation_levelこの設定はトランザクションのレコードの可視性を制御します。READ_UNCOMMITTED (isolation_level = 0) の使用により全てのレコードが見えるようになります。READ_COMMITTED (isolation_level = 1) を使うと、非トランザクションとCOMMITTEDトランザクションレコードが見えるようになります。もっと具体的には、READ_COMMITTED は現在のLSO(最後の安定したオフセット)よりも小さなオフセットから全てのデータを返し、結果の中の中断されたトランザクションのリストを含めるようにします。これによりコンシューマはABORTEDのトランザクションレコードを破棄することができます。
topicsEach topic in the request.
名前トピック名。
partitionsEach partition in the request.
partition_indexパーティションのインデックス。
timestampThe current timestamp.

ListOffsets Request (Version: 4) => replica_id isolation_level [topics] 
  replica_id => INT32
  isolation_level => INT8
  topics => name [partitions] 
    name => STRING
    partitions => partition_index current_leader_epoch timestamp 
      partition_index => INT32
      current_leader_epoch => INT32
      timestamp => INT64
フィールド 説明
replica_idThe broker ID of the requestor, or -1 if this request is being made by a normal consumer.
isolation_levelこの設定はトランザクションのレコードの可視性を制御します。READ_UNCOMMITTED (isolation_level = 0) の使用により全てのレコードが見えるようになります。READ_COMMITTED (isolation_level = 1) を使うと、非トランザクションとCOMMITTEDトランザクションレコードが見えるようになります。もっと具体的には、READ_COMMITTED は現在のLSO(最後の安定したオフセット)よりも小さなオフセットから全てのデータを返し、結果の中の中断されたトランザクションのリストを含めるようにします。これによりコンシューマはABORTEDのトランザクションレコードを破棄することができます。
topicsEach topic in the request.
名前トピック名。
partitionsEach partition in the request.
partition_indexパーティションのインデックス。
current_leader_epochThe current leader epoch.
timestampThe current timestamp.

ListOffsets Request (Version: 5) => replica_id isolation_level [topics] 
  replica_id => INT32
  isolation_level => INT8
  topics => name [partitions] 
    name => STRING
    partitions => partition_index current_leader_epoch timestamp 
      partition_index => INT32
      current_leader_epoch => INT32
      timestamp => INT64
フィールド 説明
replica_idThe broker ID of the requestor, or -1 if this request is being made by a normal consumer.
isolation_levelこの設定はトランザクションのレコードの可視性を制御します。READ_UNCOMMITTED (isolation_level = 0) の使用により全てのレコードが見えるようになります。READ_COMMITTED (isolation_level = 1) を使うと、非トランザクションとCOMMITTEDトランザクションレコードが見えるようになります。もっと具体的には、READ_COMMITTED は現在のLSO(最後の安定したオフセット)よりも小さなオフセットから全てのデータを返し、結果の中の中断されたトランザクションのリストを含めるようにします。これによりコンシューマはABORTEDのトランザクションレコードを破棄することができます。
topicsEach topic in the request.
名前トピック名。
partitionsEach partition in the request.
partition_indexパーティションのインデックス。
current_leader_epochThe current leader epoch.
timestampThe current timestamp.

Responses:

ListOffsets Response (Version: 0) => [topics] 
  topics => name [partitions] 
    name => STRING
    partitions => partition_index error_code [old_style_offsets] 
      partition_index => INT32
      error_code => INT16
      old_style_offsets => INT64
フィールド 説明
topics応答の中の各トピック。
名前トピック名。
partitionsEach partition in the response.
partition_indexパーティションのインデックス。
error_codeThe partition error code, or 0 if there was no error.
old_style_offsetsThe result offsets.

ListOffsets Response (Version: 1) => [topics] 
  topics => name [partitions] 
    name => STRING
    partitions => partition_index error_code timestamp offset 
      partition_index => INT32
      error_code => INT16
      timestamp => INT64
      offset => INT64
フィールド 説明
topics応答の中の各トピック。
名前トピック名。
partitionsEach partition in the response.
partition_indexパーティションのインデックス。
error_codeThe partition error code, or 0 if there was no error.
timestampThe timestamp associated with the returned offset.
オフセットThe returned offset.

ListOffsets Response (Version: 2) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index error_code timestamp offset 
      partition_index => INT32
      error_code => INT16
      timestamp => INT64
      offset => INT64
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
topics応答の中の各トピック。
名前トピック名。
partitionsEach partition in the response.
partition_indexパーティションのインデックス。
error_codeThe partition error code, or 0 if there was no error.
timestampThe timestamp associated with the returned offset.
オフセットThe returned offset.

ListOffsets Response (Version: 3) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index error_code timestamp offset 
      partition_index => INT32
      error_code => INT16
      timestamp => INT64
      offset => INT64
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
topics応答の中の各トピック。
名前トピック名。
partitionsEach partition in the response.
partition_indexパーティションのインデックス。
error_codeThe partition error code, or 0 if there was no error.
timestampThe timestamp associated with the returned offset.
オフセットThe returned offset.

ListOffsets Response (Version: 4) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index error_code timestamp offset leader_epoch 
      partition_index => INT32
      error_code => INT16
      timestamp => INT64
      offset => INT64
      leader_epoch => INT32
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
topics応答の中の各トピック。
名前トピック名。
partitionsEach partition in the response.
partition_indexパーティションのインデックス。
error_codeThe partition error code, or 0 if there was no error.
timestampThe timestamp associated with the returned offset.
オフセットThe returned offset.
leader_epoch

ListOffsets Response (Version: 5) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index error_code timestamp offset leader_epoch 
      partition_index => INT32
      error_code => INT16
      timestamp => INT64
      offset => INT64
      leader_epoch => INT32
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
topics応答の中の各トピック。
名前トピック名。
partitionsEach partition in the response.
partition_indexパーティションのインデックス。
error_codeThe partition error code, or 0 if there was no error.
timestampThe timestamp associated with the returned offset.
オフセットThe returned offset.
leader_epoch

Metadata API (Key: 3):
Requests:

Metadata Request (Version: 0) => [topics] 
  topics => name 
    name => STRING
フィールド 説明
topicsメタデータをフェッチするためのトピック。
名前トピック名。

Metadata Request (Version: 1) => [topics] 
  topics => name 
    name => STRING
フィールド 説明
topicsメタデータをフェッチするためのトピック。
名前トピック名。

Metadata Request (Version: 2) => [topics] 
  topics => name 
    name => STRING
フィールド 説明
topicsメタデータをフェッチするためのトピック。
名前トピック名。

Metadata Request (Version: 3) => [topics] 
  topics => name 
    name => STRING
フィールド 説明
topicsメタデータをフェッチするためのトピック。
名前トピック名。

Metadata Request (Version: 4) => [topics] allow_auto_topic_creation 
  topics => name 
    name => STRING
  allow_auto_topic_creation => BOOLEAN
フィールド 説明
topicsメタデータをフェッチするためのトピック。
名前トピック名。
allow_auto_topic_creationこれがtrueの場合、ブローカーはそうするように設定されている場合に、要求されたトピックがまだ存在しない時に自動生成するかもしれません。

Metadata Request (Version: 5) => [topics] allow_auto_topic_creation 
  topics => name 
    name => STRING
  allow_auto_topic_creation => BOOLEAN
フィールド 説明
topicsメタデータをフェッチするためのトピック。
名前トピック名。
allow_auto_topic_creationこれがtrueの場合、ブローカーはそうするように設定されている場合に、要求されたトピックがまだ存在しない時に自動生成するかもしれません。

Metadata Request (Version: 6) => [topics] allow_auto_topic_creation 
  topics => name 
    name => STRING
  allow_auto_topic_creation => BOOLEAN
フィールド 説明
topicsメタデータをフェッチするためのトピック。
名前トピック名。
allow_auto_topic_creationこれがtrueの場合、ブローカーはそうするように設定されている場合に、要求されたトピックがまだ存在しない時に自動生成するかもしれません。

Metadata Request (Version: 7) => [topics] allow_auto_topic_creation 
  topics => name 
    name => STRING
  allow_auto_topic_creation => BOOLEAN
フィールド 説明
topicsメタデータをフェッチするためのトピック。
名前トピック名。
allow_auto_topic_creationこれがtrueの場合、ブローカーはそうするように設定されている場合に、要求されたトピックがまだ存在しない時に自動生成するかもしれません。

Metadata Request (Version: 8) => [topics] allow_auto_topic_creation include_cluster_authorized_operations include_topic_authorized_operations 
  topics => name 
    name => STRING
  allow_auto_topic_creation => BOOLEAN
  include_cluster_authorized_operations => BOOLEAN
  include_topic_authorized_operations => BOOLEAN
フィールド 説明
topicsメタデータをフェッチするためのトピック。
名前トピック名。
allow_auto_topic_creationこれがtrueの場合、ブローカーはそうするように設定されている場合に、要求されたトピックがまだ存在しない時に自動生成するかもしれません。
include_cluster_authorized_operationsクラスタの認証操作を含むかどうか。
include_topic_authorized_operationsトピックの認証操作を含むかどうか。

Metadata Request (Version: 9) => [topics] allow_auto_topic_creation include_cluster_authorized_operations include_topic_authorized_operations TAG_BUFFER 
  topics => name TAG_BUFFER 
    name => COMPACT_STRING
  allow_auto_topic_creation => BOOLEAN
  include_cluster_authorized_operations => BOOLEAN
  include_topic_authorized_operations => BOOLEAN
フィールド 説明
topicsメタデータをフェッチするためのトピック。
名前トピック名。
_tagged_fieldsタグ付きのフィールド
allow_auto_topic_creationこれがtrueの場合、ブローカーはそうするように設定されている場合に、要求されたトピックがまだ存在しない時に自動生成するかもしれません。
include_cluster_authorized_operationsクラスタの認証操作を含むかどうか。
include_topic_authorized_operationsトピックの認証操作を含むかどうか。
_tagged_fieldsタグ付きのフィールド

Responses:

Metadata Response (Version: 0) => [brokers] [topics] 
  brokers => node_id host port 
    node_id => INT32
    host => STRING
    port => INT32
  topics => error_code name [partitions] 
    error_code => INT16
    name => STRING
    partitions => error_code partition_index leader_id [replica_nodes] [isr_nodes] 
      error_code => INT16
      partition_index => INT32
      leader_id => INT32
      replica_nodes => INT32
      isr_nodes => INT32
フィールド 説明
brokers応答の中の各ブローカー。
node_idブローカーのID。
hostブローカーのホスト名。
portブローカーのポート。
topics応答の中の各トピック。
error_codeトピック エラー、またはエラーが無い場合は0。
名前トピック名。
partitionsトピック内の各パーティション。
error_codeパーティション エラー、またはエラーが無い場合は0。
partition_indexパーティションのインデックス。
leader_idリーダー ブローカーのID。
replica_nodesこのパーティションをホストする全てのノードのセット。
isr_nodesこのパーティションについてリーダーと一緒に同期するノードのセット。

Metadata Response (Version: 1) => [brokers] controller_id [topics] 
  brokers => node_id host port rack 
    node_id => INT32
    host => STRING
    port => INT32
    rack => NULLABLE_STRING
  controller_id => INT32
  topics => error_code name is_internal [partitions] 
    error_code => INT16
    name => STRING
    is_internal => BOOLEAN
    partitions => error_code partition_index leader_id [replica_nodes] [isr_nodes] 
      error_code => INT16
      partition_index => INT32
      leader_id => INT32
      replica_nodes => INT32
      isr_nodes => INT32
フィールド 説明
brokers応答の中の各ブローカー。
node_idブローカーのID。
hostブローカーのホスト名。
portブローカーのポート。
rackブローカーのラック。あるいはラックに割り当てられていない場合は null。
controller_idコントローラー ブローカーのID。
topics応答の中の各トピック。
error_codeトピック エラー、またはエラーが無い場合は0。
名前トピック名。
is_internalトピックが内部的なものの場合は true。
partitionsトピック内の各パーティション。
error_codeパーティション エラー、またはエラーが無い場合は0。
partition_indexパーティションのインデックス。
leader_idリーダー ブローカーのID。
replica_nodesこのパーティションをホストする全てのノードのセット。
isr_nodesこのパーティションについてリーダーと一緒に同期するノードのセット。

Metadata Response (Version: 2) => [brokers] cluster_id controller_id [topics] 
  brokers => node_id host port rack 
    node_id => INT32
    host => STRING
    port => INT32
    rack => NULLABLE_STRING
  cluster_id => NULLABLE_STRING
  controller_id => INT32
  topics => error_code name is_internal [partitions] 
    error_code => INT16
    name => STRING
    is_internal => BOOLEAN
    partitions => error_code partition_index leader_id [replica_nodes] [isr_nodes] 
      error_code => INT16
      partition_index => INT32
      leader_id => INT32
      replica_nodes => INT32
      isr_nodes => INT32
フィールド 説明
brokers応答の中の各ブローカー。
node_idブローカーのID。
hostブローカーのホスト名。
portブローカーのポート。
rackブローカーのラック。あるいはラックに割り当てられていない場合は null。
cluster_id対応するブルーカーが所属するクラスタID。
controller_idコントローラー ブローカーのID。
topics応答の中の各トピック。
error_codeトピック エラー、またはエラーが無い場合は0。
名前トピック名。
is_internalトピックが内部的なものの場合は true。
partitionsトピック内の各パーティション。
error_codeパーティション エラー、またはエラーが無い場合は0。
partition_indexパーティションのインデックス。
leader_idリーダー ブローカーのID。
replica_nodesこのパーティションをホストする全てのノードのセット。
isr_nodesこのパーティションについてリーダーと一緒に同期するノードのセット。

Metadata Response (Version: 3) => throttle_time_ms [brokers] cluster_id controller_id [topics] 
  throttle_time_ms => INT32
  brokers => node_id host port rack 
    node_id => INT32
    host => STRING
    port => INT32
    rack => NULLABLE_STRING
  cluster_id => NULLABLE_STRING
  controller_id => INT32
  topics => error_code name is_internal [partitions] 
    error_code => INT16
    name => STRING
    is_internal => BOOLEAN
    partitions => error_code partition_index leader_id [replica_nodes] [isr_nodes] 
      error_code => INT16
      partition_index => INT32
      leader_id => INT32
      replica_nodes => INT32
      isr_nodes => INT32
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
brokers応答の中の各ブローカー。
node_idブローカーのID。
hostブローカーのホスト名。
portブローカーのポート。
rackブローカーのラック。あるいはラックに割り当てられていない場合は null。
cluster_id対応するブルーカーが所属するクラスタID。
controller_idコントローラー ブローカーのID。
topics応答の中の各トピック。
error_codeトピック エラー、またはエラーが無い場合は0。
名前トピック名。
is_internalトピックが内部的なものの場合は true。
partitionsトピック内の各パーティション。
error_codeパーティション エラー、またはエラーが無い場合は0。
partition_indexパーティションのインデックス。
leader_idリーダー ブローカーのID。
replica_nodesこのパーティションをホストする全てのノードのセット。
isr_nodesこのパーティションについてリーダーと一緒に同期するノードのセット。

Metadata Response (Version: 4) => throttle_time_ms [brokers] cluster_id controller_id [topics] 
  throttle_time_ms => INT32
  brokers => node_id host port rack 
    node_id => INT32
    host => STRING
    port => INT32
    rack => NULLABLE_STRING
  cluster_id => NULLABLE_STRING
  controller_id => INT32
  topics => error_code name is_internal [partitions] 
    error_code => INT16
    name => STRING
    is_internal => BOOLEAN
    partitions => error_code partition_index leader_id [replica_nodes] [isr_nodes] 
      error_code => INT16
      partition_index => INT32
      leader_id => INT32
      replica_nodes => INT32
      isr_nodes => INT32
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
brokers応答の中の各ブローカー。
node_idブローカーのID。
hostブローカーのホスト名。
portブローカーのポート。
rackブローカーのラック。あるいはラックに割り当てられていない場合は null。
cluster_id対応するブルーカーが所属するクラスタID。
controller_idコントローラー ブローカーのID。
topics応答の中の各トピック。
error_codeトピック エラー、またはエラーが無い場合は0。
名前トピック名。
is_internalトピックが内部的なものの場合は true。
partitionsトピック内の各パーティション。
error_codeパーティション エラー、またはエラーが無い場合は0。
partition_indexパーティションのインデックス。
leader_idリーダー ブローカーのID。
replica_nodesこのパーティションをホストする全てのノードのセット。
isr_nodesこのパーティションについてリーダーと一緒に同期するノードのセット。

Metadata Response (Version: 5) => throttle_time_ms [brokers] cluster_id controller_id [topics] 
  throttle_time_ms => INT32
  brokers => node_id host port rack 
    node_id => INT32
    host => STRING
    port => INT32
    rack => NULLABLE_STRING
  cluster_id => NULLABLE_STRING
  controller_id => INT32
  topics => error_code name is_internal [partitions] 
    error_code => INT16
    name => STRING
    is_internal => BOOLEAN
    partitions => error_code partition_index leader_id [replica_nodes] [isr_nodes] [offline_replicas] 
      error_code => INT16
      partition_index => INT32
      leader_id => INT32
      replica_nodes => INT32
      isr_nodes => INT32
      offline_replicas => INT32
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
brokers応答の中の各ブローカー。
node_idブローカーのID。
hostブローカーのホスト名。
portブローカーのポート。
rackブローカーのラック。あるいはラックに割り当てられていない場合は null。
cluster_id対応するブルーカーが所属するクラスタID。
controller_idコントローラー ブローカーのID。
topics応答の中の各トピック。
error_codeトピック エラー、またはエラーが無い場合は0。
名前トピック名。
is_internalトピックが内部的なものの場合は true。
partitionsトピック内の各パーティション。
error_codeパーティション エラー、またはエラーが無い場合は0。
partition_indexパーティションのインデックス。
leader_idリーダー ブローカーのID。
replica_nodesこのパーティションをホストする全てのノードのセット。
isr_nodesこのパーティションについてリーダーと一緒に同期するノードのセット。
offline_replicasこのパーティションのオフライン レプリカのセット。

Metadata Response (Version: 6) => throttle_time_ms [brokers] cluster_id controller_id [topics] 
  throttle_time_ms => INT32
  brokers => node_id host port rack 
    node_id => INT32
    host => STRING
    port => INT32
    rack => NULLABLE_STRING
  cluster_id => NULLABLE_STRING
  controller_id => INT32
  topics => error_code name is_internal [partitions] 
    error_code => INT16
    name => STRING
    is_internal => BOOLEAN
    partitions => error_code partition_index leader_id [replica_nodes] [isr_nodes] [offline_replicas] 
      error_code => INT16
      partition_index => INT32
      leader_id => INT32
      replica_nodes => INT32
      isr_nodes => INT32
      offline_replicas => INT32
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
brokers応答の中の各ブローカー。
node_idブローカーのID。
hostブローカーのホスト名。
portブローカーのポート。
rackブローカーのラック。あるいはラックに割り当てられていない場合は null。
cluster_id対応するブルーカーが所属するクラスタID。
controller_idコントローラー ブローカーのID。
topics応答の中の各トピック。
error_codeトピック エラー、またはエラーが無い場合は0。
名前トピック名。
is_internalトピックが内部的なものの場合は true。
partitionsトピック内の各パーティション。
error_codeパーティション エラー、またはエラーが無い場合は0。
partition_indexパーティションのインデックス。
leader_idリーダー ブローカーのID。
replica_nodesこのパーティションをホストする全てのノードのセット。
isr_nodesこのパーティションについてリーダーと一緒に同期するノードのセット。
offline_replicasこのパーティションのオフライン レプリカのセット。

Metadata Response (Version: 7) => throttle_time_ms [brokers] cluster_id controller_id [topics] 
  throttle_time_ms => INT32
  brokers => node_id host port rack 
    node_id => INT32
    host => STRING
    port => INT32
    rack => NULLABLE_STRING
  cluster_id => NULLABLE_STRING
  controller_id => INT32
  topics => error_code name is_internal [partitions] 
    error_code => INT16
    name => STRING
    is_internal => BOOLEAN
    partitions => error_code partition_index leader_id leader_epoch [replica_nodes] [isr_nodes] [offline_replicas] 
      error_code => INT16
      partition_index => INT32
      leader_id => INT32
      leader_epoch => INT32
      replica_nodes => INT32
      isr_nodes => INT32
      offline_replicas => INT32
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
brokers応答の中の各ブローカー。
node_idブローカーのID。
hostブローカーのホスト名。
portブローカーのポート。
rackブローカーのラック。あるいはラックに割り当てられていない場合は null。
cluster_id対応するブルーカーが所属するクラスタID。
controller_idコントローラー ブローカーのID。
topics応答の中の各トピック。
error_codeトピック エラー、またはエラーが無い場合は0。
名前トピック名。
is_internalトピックが内部的なものの場合は true。
partitionsトピック内の各パーティション。
error_codeパーティション エラー、またはエラーが無い場合は0。
partition_indexパーティションのインデックス。
leader_idリーダー ブローカーのID。
leader_epochこのパーティションのリーダー エポック。
replica_nodesこのパーティションをホストする全てのノードのセット。
isr_nodesこのパーティションについてリーダーと一緒に同期するノードのセット。
offline_replicasこのパーティションのオフライン レプリカのセット。

Metadata Response (Version: 8) => throttle_time_ms [brokers] cluster_id controller_id [topics] cluster_authorized_operations 
  throttle_time_ms => INT32
  brokers => node_id host port rack 
    node_id => INT32
    host => STRING
    port => INT32
    rack => NULLABLE_STRING
  cluster_id => NULLABLE_STRING
  controller_id => INT32
  topics => error_code name is_internal [partitions] topic_authorized_operations 
    error_code => INT16
    name => STRING
    is_internal => BOOLEAN
    partitions => error_code partition_index leader_id leader_epoch [replica_nodes] [isr_nodes] [offline_replicas] 
      error_code => INT16
      partition_index => INT32
      leader_id => INT32
      leader_epoch => INT32
      replica_nodes => INT32
      isr_nodes => INT32
      offline_replicas => INT32
    topic_authorized_operations => INT32
  cluster_authorized_operations => INT32
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
brokers応答の中の各ブローカー。
node_idブローカーのID。
hostブローカーのホスト名。
portブローカーのポート。
rackブローカーのラック。あるいはラックに割り当てられていない場合は null。
cluster_id対応するブルーカーが所属するクラスタID。
controller_idコントローラー ブローカーのID。
topics応答の中の各トピック。
error_codeトピック エラー、またはエラーが無い場合は0。
名前トピック名。
is_internalトピックが内部的なものの場合は true。
partitionsトピック内の各パーティション。
error_codeパーティション エラー、またはエラーが無い場合は0。
partition_indexパーティションのインデックス。
leader_idリーダー ブローカーのID。
leader_epochこのパーティションのリーダー エポック。
replica_nodesこのパーティションをホストする全てのノードのセット。
isr_nodesこのパーティションについてリーダーと一緒に同期するノードのセット。
offline_replicasこのパーティションのオフライン レプリカのセット。
topic_authorized_operationsこのトピックについて認証された操作を表す32ビットのビットフィールド。
cluster_authorized_operationsこのクラスタについて認証された操作を表す32ビットのビットフィールド。

Metadata Response (Version: 9) => throttle_time_ms [brokers] cluster_id controller_id [topics] cluster_authorized_operations TAG_BUFFER 
  throttle_time_ms => INT32
  brokers => node_id host port rack TAG_BUFFER 
    node_id => INT32
    host => COMPACT_STRING
    port => INT32
    rack => COMPACT_NULLABLE_STRING
  cluster_id => COMPACT_NULLABLE_STRING
  controller_id => INT32
  topics => error_code name is_internal [partitions] topic_authorized_operations TAG_BUFFER 
    error_code => INT16
    name => COMPACT_STRING
    is_internal => BOOLEAN
    partitions => error_code partition_index leader_id leader_epoch [replica_nodes] [isr_nodes] [offline_replicas] TAG_BUFFER 
      error_code => INT16
      partition_index => INT32
      leader_id => INT32
      leader_epoch => INT32
      replica_nodes => INT32
      isr_nodes => INT32
      offline_replicas => INT32
    topic_authorized_operations => INT32
  cluster_authorized_operations => INT32
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
brokers応答の中の各ブローカー。
node_idブローカーのID。
hostブローカーのホスト名。
portブローカーのポート。
rackブローカーのラック。あるいはラックに割り当てられていない場合は null。
_tagged_fieldsタグ付きのフィールド
cluster_id対応するブルーカーが所属するクラスタID。
controller_idコントローラー ブローカーのID。
topics応答の中の各トピック。
error_codeトピック エラー、またはエラーが無い場合は0。
名前トピック名。
is_internalトピックが内部的なものの場合は true。
partitionsトピック内の各パーティション。
error_codeパーティション エラー、またはエラーが無い場合は0。
partition_indexパーティションのインデックス。
leader_idリーダー ブローカーのID。
leader_epochこのパーティションのリーダー エポック。
replica_nodesこのパーティションをホストする全てのノードのセット。
isr_nodesこのパーティションについてリーダーと一緒に同期するノードのセット。
offline_replicasこのパーティションのオフライン レプリカのセット。
_tagged_fieldsタグ付きのフィールド
topic_authorized_operationsこのトピックについて認証された操作を表す32ビットのビットフィールド。
_tagged_fieldsタグ付きのフィールド
cluster_authorized_operationsこのクラスタについて認証された操作を表す32ビットのビットフィールド。
_tagged_fieldsタグ付きのフィールド

LeaderAndIsr API (Key: 4):
Requests:

LeaderAndIsr Request (Version: 0) => controller_id controller_epoch [ungrouped_partition_states] [live_leaders] 
  controller_id => INT32
  controller_epoch => INT32
  ungrouped_partition_states => topic_name partition_index controller_epoch leader leader_epoch [isr] zk_version [replicas] 
    topic_name => STRING
    partition_index => INT32
    controller_epoch => INT32
    leader => INT32
    leader_epoch => INT32
    isr => INT32
    zk_version => INT32
    replicas => INT32
  live_leaders => broker_id host_name port 
    broker_id => INT32
    host_name => STRING
    port => INT32
フィールド 説明
controller_id現在のコントローラ ID。
controller_epoch現在のコントローラのエポック。
ungrouped_partition_statesv0 または v1 メッセージ内の各パーティションの状態。
topic_nameトピック名。これは、v0 または v1 にのみ存在します。
partition_indexパーティションのインデックス。
controller_epochコントローラー epoch。
leaderリーダーのブローカー ID。
leader_epochリーダーのepoch。
isr同期中のレプリカ ID。
zk_versionZooKeeper のバージョン。
replicasレプリカ ID。
live_leaders現在のライブリーダー。
broker_idリーダーのブローカー ID。
host_nameリーダーのホスト名。
portリーダーのポート。

LeaderAndIsr Request (Version: 1) => controller_id controller_epoch [ungrouped_partition_states] [live_leaders] 
  controller_id => INT32
  controller_epoch => INT32
  ungrouped_partition_states => topic_name partition_index controller_epoch leader leader_epoch [isr] zk_version [replicas] is_new 
    topic_name => STRING
    partition_index => INT32
    controller_epoch => INT32
    leader => INT32
    leader_epoch => INT32
    isr => INT32
    zk_version => INT32
    replicas => INT32
    is_new => BOOLEAN
  live_leaders => broker_id host_name port 
    broker_id => INT32
    host_name => STRING
    port => INT32
フィールド 説明
controller_id現在のコントローラ ID。
controller_epoch現在のコントローラのエポック。
ungrouped_partition_statesv0 または v1 メッセージ内の各パーティションの状態。
topic_nameトピック名。これは、v0 または v1 にのみ存在します。
partition_indexパーティションのインデックス。
controller_epochコントローラー epoch。
leaderリーダーのブローカー ID。
leader_epochリーダーのepoch。
isr同期中のレプリカ ID。
zk_versionZooKeeper のバージョン。
replicasレプリカ ID。
is_newレプリカがブローカー上に存在しなければならないかどうか。
live_leaders現在のライブリーダー。
broker_idリーダーのブローカー ID。
host_nameリーダーのホスト名。
portリーダーのポート。

LeaderAndIsr Request (Version: 2) => controller_id controller_epoch broker_epoch [topic_states] [live_leaders] 
  controller_id => INT32
  controller_epoch => INT32
  broker_epoch => INT64
  topic_states => topic_name [partition_states] 
    topic_name => STRING
    partition_states => partition_index controller_epoch leader leader_epoch [isr] zk_version [replicas] is_new 
      partition_index => INT32
      controller_epoch => INT32
      leader => INT32
      leader_epoch => INT32
      isr => INT32
      zk_version => INT32
      replicas => INT32
      is_new => BOOLEAN
  live_leaders => broker_id host_name port 
    broker_id => INT32
    host_name => STRING
    port => INT32
フィールド 説明
controller_id現在のコントローラ ID。
controller_epoch現在のコントローラのエポック。
broker_epoch現在のブローカーのエポック。
topic_states各トピック。
topic_nameトピック名。
partition_states各パーティションの状態
partition_indexパーティションのインデックス。
controller_epochコントローラー epoch。
leaderリーダーのブローカー ID。
leader_epochリーダーのepoch。
isr同期中のレプリカ ID。
zk_versionZooKeeper のバージョン。
replicasレプリカ ID。
is_newレプリカがブローカー上に存在しなければならないかどうか。
live_leaders現在のライブリーダー。
broker_idリーダーのブローカー ID。
host_nameリーダーのホスト名。
portリーダーのポート。

LeaderAndIsr Request (Version: 3) => controller_id controller_epoch broker_epoch [topic_states] [live_leaders] 
  controller_id => INT32
  controller_epoch => INT32
  broker_epoch => INT64
  topic_states => topic_name [partition_states] 
    topic_name => STRING
    partition_states => partition_index controller_epoch leader leader_epoch [isr] zk_version [replicas] [adding_replicas] [removing_replicas] is_new 
      partition_index => INT32
      controller_epoch => INT32
      leader => INT32
      leader_epoch => INT32
      isr => INT32
      zk_version => INT32
      replicas => INT32
      adding_replicas => INT32
      removing_replicas => INT32
      is_new => BOOLEAN
  live_leaders => broker_id host_name port 
    broker_id => INT32
    host_name => STRING
    port => INT32
フィールド 説明
controller_id現在のコントローラ ID。
controller_epoch現在のコントローラのエポック。
broker_epoch現在のブローカーのエポック。
topic_states各トピック。
topic_nameトピック名。
partition_states各パーティションの状態
partition_indexパーティションのインデックス。
controller_epochコントローラー epoch。
leaderリーダーのブローカー ID。
leader_epochリーダーのepoch。
isr同期中のレプリカ ID。
zk_versionZooKeeper のバージョン。
replicasレプリカ ID。
adding_replicasこのパーティションを追加するレプリカ ID、あるいはレプリカが追加されていない場合は null。
removing_replicasこのパーティションを削除するレプリカ ID、あるいはレプリカが削除されない場合は null。
is_newレプリカがブローカー上に存在しなければならないかどうか。
live_leaders現在のライブリーダー。
broker_idリーダーのブローカー ID。
host_nameリーダーのホスト名。
portリーダーのポート。

LeaderAndIsr Request (Version: 4) => controller_id controller_epoch broker_epoch [topic_states] [live_leaders] TAG_BUFFER 
  controller_id => INT32
  controller_epoch => INT32
  broker_epoch => INT64
  topic_states => topic_name [partition_states] TAG_BUFFER 
    topic_name => COMPACT_STRING
    partition_states => partition_index controller_epoch leader leader_epoch [isr] zk_version [replicas] [adding_replicas] [removing_replicas] is_new TAG_BUFFER 
      partition_index => INT32
      controller_epoch => INT32
      leader => INT32
      leader_epoch => INT32
      isr => INT32
      zk_version => INT32
      replicas => INT32
      adding_replicas => INT32
      removing_replicas => INT32
      is_new => BOOLEAN
  live_leaders => broker_id host_name port TAG_BUFFER 
    broker_id => INT32
    host_name => COMPACT_STRING
    port => INT32
フィールド 説明
controller_id現在のコントローラ ID。
controller_epoch現在のコントローラのエポック。
broker_epoch現在のブローカーのエポック。
topic_states各トピック。
topic_nameトピック名。
partition_states各パーティションの状態
partition_indexパーティションのインデックス。
controller_epochコントローラー epoch。
leaderリーダーのブローカー ID。
leader_epochリーダーのepoch。
isr同期中のレプリカ ID。
zk_versionZooKeeper のバージョン。
replicasレプリカ ID。
adding_replicasこのパーティションを追加するレプリカ ID、あるいはレプリカが追加されていない場合は null。
removing_replicasこのパーティションを削除するレプリカ ID、あるいはレプリカが削除されない場合は null。
is_newレプリカがブローカー上に存在しなければならないかどうか。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド
live_leaders現在のライブリーダー。
broker_idリーダーのブローカー ID。
host_nameリーダーのホスト名。
portリーダーのポート。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

Responses:

LeaderAndIsr Response (Version: 0) => error_code [partition_errors] 
  error_code => INT16
  partition_errors => topic_name partition_index error_code 
    topic_name => STRING
    partition_index => INT32
    error_code => INT16
フィールド 説明
error_codeエラーコード。あるいはエラーが無かった場合は0。
partition_errorsEach partition.
topic_nameトピック名。
partition_indexパーティションのインデックス。
error_codeThe partition error code, or 0 if there was no error.

LeaderAndIsr Response (Version: 1) => error_code [partition_errors] 
  error_code => INT16
  partition_errors => topic_name partition_index error_code 
    topic_name => STRING
    partition_index => INT32
    error_code => INT16
フィールド 説明
error_codeエラーコード。あるいはエラーが無かった場合は0。
partition_errorsEach partition.
topic_nameトピック名。
partition_indexパーティションのインデックス。
error_codeThe partition error code, or 0 if there was no error.

LeaderAndIsr Response (Version: 2) => error_code [partition_errors] 
  error_code => INT16
  partition_errors => topic_name partition_index error_code 
    topic_name => STRING
    partition_index => INT32
    error_code => INT16
フィールド 説明
error_codeエラーコード。あるいはエラーが無かった場合は0。
partition_errorsEach partition.
topic_nameトピック名。
partition_indexパーティションのインデックス。
error_codeThe partition error code, or 0 if there was no error.

LeaderAndIsr Response (Version: 3) => error_code [partition_errors] 
  error_code => INT16
  partition_errors => topic_name partition_index error_code 
    topic_name => STRING
    partition_index => INT32
    error_code => INT16
フィールド 説明
error_codeエラーコード。あるいはエラーが無かった場合は0。
partition_errorsEach partition.
topic_nameトピック名。
partition_indexパーティションのインデックス。
error_codeThe partition error code, or 0 if there was no error.

LeaderAndIsr Response (Version: 4) => error_code [partition_errors] TAG_BUFFER 
  error_code => INT16
  partition_errors => topic_name partition_index error_code TAG_BUFFER 
    topic_name => COMPACT_STRING
    partition_index => INT32
    error_code => INT16
フィールド 説明
error_codeエラーコード。あるいはエラーが無かった場合は0。
partition_errorsEach partition.
topic_nameトピック名。
partition_indexパーティションのインデックス。
error_codeThe partition error code, or 0 if there was no error.
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

StopReplica API (Key: 5):
Requests:

StopReplica Request (Version: 0) => controller_id controller_epoch delete_partitions [ungrouped_partitions] 
  controller_id => INT32
  controller_epoch => INT32
  delete_partitions => BOOLEAN
  ungrouped_partitions => topic_name partition_index 
    topic_name => STRING
    partition_index => INT32
フィールド 説明
controller_idコントローラid。
controller_epochコントローラー epoch。
delete_partitionsこれらのパーティションが削除されるべきかどうか。
ungrouped_partitions停止するパーティション。
topic_nameトピック名。
partition_indexパーティションのインデックス。

StopReplica Request (Version: 1) => controller_id controller_epoch broker_epoch delete_partitions [topics] 
  controller_id => INT32
  controller_epoch => INT32
  broker_epoch => INT64
  delete_partitions => BOOLEAN
  topics => name [partition_indexes] 
    name => STRING
    partition_indexes => INT32
フィールド 説明
controller_idコントローラid。
controller_epochコントローラー epoch。
broker_epochブローカーのエポック。
delete_partitionsこれらのパーティションが削除されるべきかどうか。
topics停止するトピック。
名前トピック名。
partition_indexesパーティションのインデックス。

StopReplica Request (Version: 2) => controller_id controller_epoch broker_epoch delete_partitions [topics] TAG_BUFFER 
  controller_id => INT32
  controller_epoch => INT32
  broker_epoch => INT64
  delete_partitions => BOOLEAN
  topics => name [partition_indexes] TAG_BUFFER 
    name => COMPACT_STRING
    partition_indexes => INT32
フィールド 説明
controller_idコントローラid。
controller_epochコントローラー epoch。
broker_epochブローカーのエポック。
delete_partitionsこれらのパーティションが削除されるべきかどうか。
topics停止するトピック。
名前トピック名。
partition_indexesパーティションのインデックス。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

StopReplica Request (Version: 3) => controller_id controller_epoch broker_epoch [topic_states] TAG_BUFFER 
  controller_id => INT32
  controller_epoch => INT32
  broker_epoch => INT64
  topic_states => topic_name [partition_states] TAG_BUFFER 
    topic_name => COMPACT_STRING
    partition_states => partition_index leader_epoch delete_partition TAG_BUFFER 
      partition_index => INT32
      leader_epoch => INT32
      delete_partition => BOOLEAN
フィールド 説明
controller_idコントローラid。
controller_epochコントローラー epoch。
broker_epochブローカーのエポック。
topic_states各トピック。
topic_nameトピック名。
partition_states各パーティションの状態
partition_indexパーティションのインデックス。
leader_epochリーダーのepoch。
delete_partitionこのパーティションが削除されるべきかどうか。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

Responses:

StopReplica Response (Version: 0) => error_code [partition_errors] 
  error_code => INT16
  partition_errors => topic_name partition_index error_code 
    topic_name => STRING
    partition_index => INT32
    error_code => INT16
フィールド 説明
error_codeトップレベルのエラーコード。トップレベルのエラーが無い場合は 0。
partition_errors各パーティションについての応答。
topic_nameトピック名。
partition_indexパーティションのインデックス。
error_codeパーティションエラーコード。またはパーティションエラーが無い場合は 0。

StopReplica Response (Version: 1) => error_code [partition_errors] 
  error_code => INT16
  partition_errors => topic_name partition_index error_code 
    topic_name => STRING
    partition_index => INT32
    error_code => INT16
フィールド 説明
error_codeトップレベルのエラーコード。トップレベルのエラーが無い場合は 0。
partition_errors各パーティションについての応答。
topic_nameトピック名。
partition_indexパーティションのインデックス。
error_codeパーティションエラーコード。またはパーティションエラーが無い場合は 0。

StopReplica Response (Version: 2) => error_code [partition_errors] TAG_BUFFER 
  error_code => INT16
  partition_errors => topic_name partition_index error_code TAG_BUFFER 
    topic_name => COMPACT_STRING
    partition_index => INT32
    error_code => INT16
フィールド 説明
error_codeトップレベルのエラーコード。トップレベルのエラーが無い場合は 0。
partition_errors各パーティションについての応答。
topic_nameトピック名。
partition_indexパーティションのインデックス。
error_codeパーティションエラーコード。またはパーティションエラーが無い場合は 0。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

StopReplica Response (Version: 3) => error_code [partition_errors] TAG_BUFFER 
  error_code => INT16
  partition_errors => topic_name partition_index error_code TAG_BUFFER 
    topic_name => COMPACT_STRING
    partition_index => INT32
    error_code => INT16
フィールド 説明
error_codeトップレベルのエラーコード。トップレベルのエラーが無い場合は 0。
partition_errors各パーティションについての応答。
topic_nameトピック名。
partition_indexパーティションのインデックス。
error_codeパーティションエラーコード。またはパーティションエラーが無い場合は 0。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

UpdateMetadata API (Key: 6):
Requests:

UpdateMetadata Request (Version: 0) => controller_id controller_epoch [ungrouped_partition_states] [live_brokers] 
  controller_id => INT32
  controller_epoch => INT32
  ungrouped_partition_states => topic_name partition_index controller_epoch leader leader_epoch [isr] zk_version [replicas] 
    topic_name => STRING
    partition_index => INT32
    controller_epoch => INT32
    leader => INT32
    leader_epoch => INT32
    isr => INT32
    zk_version => INT32
    replicas => INT32
  live_brokers => id v0_host v0_port 
    id => INT32
    v0_host => STRING
    v0_port => INT32
フィールド 説明
controller_idコントローラid。
controller_epochコントローラー epoch。
ungrouped_partition_statesこの RPC の古いバージョンでは、更新する各パーティション。
topic_nameこの RPC の古いバージョンでは、トピック名。
partition_indexパーティションのインデックス。
controller_epochコントローラー epoch。
leader現在のパーティションリーダーであるブローカーの ID。
leader_epochこのパーティションのリーダー エポック。
isrこのパーティションの ISR にあるブローカー。
zk_versionZookeeper のバージョン。
replicasこのパーティションの全てのレプリカ。
live_brokers
idブローカーid。
v0_hostブローカーのホスト名。
v0_portブローカーのポート。

UpdateMetadata Request (Version: 1) => controller_id controller_epoch [ungrouped_partition_states] [live_brokers] 
  controller_id => INT32
  controller_epoch => INT32
  ungrouped_partition_states => topic_name partition_index controller_epoch leader leader_epoch [isr] zk_version [replicas] 
    topic_name => STRING
    partition_index => INT32
    controller_epoch => INT32
    leader => INT32
    leader_epoch => INT32
    isr => INT32
    zk_version => INT32
    replicas => INT32
  live_brokers => id [endpoints] 
    id => INT32
    endpoints => port host security_protocol 
      port => INT32
      host => STRING
      security_protocol => INT16
フィールド 説明
controller_idコントローラid。
controller_epochコントローラー epoch。
ungrouped_partition_statesこの RPC の古いバージョンでは、更新する各パーティション。
topic_nameこの RPC の古いバージョンでは、トピック名。
partition_indexパーティションのインデックス。
controller_epochコントローラー epoch。
leader現在のパーティションリーダーであるブローカーの ID。
leader_epochこのパーティションのリーダー エポック。
isrこのパーティションの ISR にあるブローカー。
zk_versionZookeeper のバージョン。
replicasこのパーティションの全てのレプリカ。
live_brokers
idブローカーid。
endpointsブローカーのエンドポイント。The broker endpoints.
portこのエンドポイントのポート。
hostこのエンドポイントのホスト名。
security_protocolセキュリティ プロトコルの型

UpdateMetadata Request (Version: 2) => controller_id controller_epoch [ungrouped_partition_states] [live_brokers] 
  controller_id => INT32
  controller_epoch => INT32
  ungrouped_partition_states => topic_name partition_index controller_epoch leader leader_epoch [isr] zk_version [replicas] 
    topic_name => STRING
    partition_index => INT32
    controller_epoch => INT32
    leader => INT32
    leader_epoch => INT32
    isr => INT32
    zk_version => INT32
    replicas => INT32
  live_brokers => id [endpoints] rack 
    id => INT32
    endpoints => port host security_protocol 
      port => INT32
      host => STRING
      security_protocol => INT16
    rack => NULLABLE_STRING
フィールド 説明
controller_idコントローラid。
controller_epochコントローラー epoch。
ungrouped_partition_statesこの RPC の古いバージョンでは、更新する各パーティション。
topic_nameこの RPC の古いバージョンでは、トピック名。
partition_indexパーティションのインデックス。
controller_epochコントローラー epoch。
leader現在のパーティションリーダーであるブローカーの ID。
leader_epochこのパーティションのリーダー エポック。
isrこのパーティションの ISR にあるブローカー。
zk_versionZookeeper のバージョン。
replicasこのパーティションの全てのレプリカ。
live_brokers
idブローカーid。
endpointsブローカーのエンドポイント。The broker endpoints.
portこのエンドポイントのポート。
hostこのエンドポイントのホスト名。
security_protocolセキュリティ プロトコルの型
rackこのブローカーが所属するラック。

UpdateMetadata Request (Version: 3) => controller_id controller_epoch [ungrouped_partition_states] [live_brokers] 
  controller_id => INT32
  controller_epoch => INT32
  ungrouped_partition_states => topic_name partition_index controller_epoch leader leader_epoch [isr] zk_version [replicas] 
    topic_name => STRING
    partition_index => INT32
    controller_epoch => INT32
    leader => INT32
    leader_epoch => INT32
    isr => INT32
    zk_version => INT32
    replicas => INT32
  live_brokers => id [endpoints] rack 
    id => INT32
    endpoints => port host listener security_protocol 
      port => INT32
      host => STRING
      listener => STRING
      security_protocol => INT16
    rack => NULLABLE_STRING
フィールド 説明
controller_idコントローラid。
controller_epochコントローラー epoch。
ungrouped_partition_statesこの RPC の古いバージョンでは、更新する各パーティション。
topic_nameこの RPC の古いバージョンでは、トピック名。
partition_indexパーティションのインデックス。
controller_epochコントローラー epoch。
leader現在のパーティションリーダーであるブローカーの ID。
leader_epochこのパーティションのリーダー エポック。
isrこのパーティションの ISR にあるブローカー。
zk_versionZookeeper のバージョン。
replicasこのパーティションの全てのレプリカ。
live_brokers
idブローカーid。
endpointsブローカーのエンドポイント。The broker endpoints.
portこのエンドポイントのポート。
hostこのエンドポイントのホスト名。
listenerリスナー名。
security_protocolセキュリティ プロトコルの型
rackこのブローカーが所属するラック。

UpdateMetadata Request (Version: 4) => controller_id controller_epoch [ungrouped_partition_states] [live_brokers] 
  controller_id => INT32
  controller_epoch => INT32
  ungrouped_partition_states => topic_name partition_index controller_epoch leader leader_epoch [isr] zk_version [replicas] [offline_replicas] 
    topic_name => STRING
    partition_index => INT32
    controller_epoch => INT32
    leader => INT32
    leader_epoch => INT32
    isr => INT32
    zk_version => INT32
    replicas => INT32
    offline_replicas => INT32
  live_brokers => id [endpoints] rack 
    id => INT32
    endpoints => port host listener security_protocol 
      port => INT32
      host => STRING
      listener => STRING
      security_protocol => INT16
    rack => NULLABLE_STRING
フィールド 説明
controller_idコントローラid。
controller_epochコントローラー epoch。
ungrouped_partition_statesこの RPC の古いバージョンでは、更新する各パーティション。
topic_nameこの RPC の古いバージョンでは、トピック名。
partition_indexパーティションのインデックス。
controller_epochコントローラー epoch。
leader現在のパーティションリーダーであるブローカーの ID。
leader_epochこのパーティションのリーダー エポック。
isrこのパーティションの ISR にあるブローカー。
zk_versionZookeeper のバージョン。
replicasこのパーティションの全てのレプリカ。
offline_replicasオフラインのこのパーティションのレプリカ。
live_brokers
idブローカーid。
endpointsブローカーのエンドポイント。The broker endpoints.
portこのエンドポイントのポート。
hostこのエンドポイントのホスト名。
listenerリスナー名。
security_protocolセキュリティ プロトコルの型
rackこのブローカーが所属するラック。

UpdateMetadata Request (Version: 5) => controller_id controller_epoch broker_epoch [topic_states] [live_brokers] 
  controller_id => INT32
  controller_epoch => INT32
  broker_epoch => INT64
  topic_states => topic_name [partition_states] 
    topic_name => STRING
    partition_states => partition_index controller_epoch leader leader_epoch [isr] zk_version [replicas] [offline_replicas] 
      partition_index => INT32
      controller_epoch => INT32
      leader => INT32
      leader_epoch => INT32
      isr => INT32
      zk_version => INT32
      replicas => INT32
      offline_replicas => INT32
  live_brokers => id [endpoints] rack 
    id => INT32
    endpoints => port host listener security_protocol 
      port => INT32
      host => STRING
      listener => STRING
      security_protocol => INT16
    rack => NULLABLE_STRING
フィールド 説明
controller_idコントローラid。
controller_epochコントローラー epoch。
broker_epochブローカーのエポック。
topic_statesこの RPC の新しいバージョンでは、更新される各トピック。
topic_nameトピック名。
partition_states更新されるパーティション。
partition_indexパーティションのインデックス。
controller_epochコントローラー epoch。
leader現在のパーティションリーダーであるブローカーの ID。
leader_epochこのパーティションのリーダー エポック。
isrこのパーティションの ISR にあるブローカー。
zk_versionZookeeper のバージョン。
replicasこのパーティションの全てのレプリカ。
offline_replicasオフラインのこのパーティションのレプリカ。
live_brokers
idブローカーid。
endpointsブローカーのエンドポイント。The broker endpoints.
portこのエンドポイントのポート。
hostこのエンドポイントのホスト名。
listenerリスナー名。
security_protocolセキュリティ プロトコルの型
rackこのブローカーが所属するラック。

UpdateMetadata Request (Version: 6) => controller_id controller_epoch broker_epoch [topic_states] [live_brokers] TAG_BUFFER 
  controller_id => INT32
  controller_epoch => INT32
  broker_epoch => INT64
  topic_states => topic_name [partition_states] TAG_BUFFER 
    topic_name => COMPACT_STRING
    partition_states => partition_index controller_epoch leader leader_epoch [isr] zk_version [replicas] [offline_replicas] TAG_BUFFER 
      partition_index => INT32
      controller_epoch => INT32
      leader => INT32
      leader_epoch => INT32
      isr => INT32
      zk_version => INT32
      replicas => INT32
      offline_replicas => INT32
  live_brokers => id [endpoints] rack TAG_BUFFER 
    id => INT32
    endpoints => port host listener security_protocol TAG_BUFFER 
      port => INT32
      host => COMPACT_STRING
      listener => COMPACT_STRING
      security_protocol => INT16
    rack => COMPACT_NULLABLE_STRING
フィールド 説明
controller_idコントローラid。
controller_epochコントローラー epoch。
broker_epochブローカーのエポック。
topic_statesこの RPC の新しいバージョンでは、更新される各トピック。
topic_nameトピック名。
partition_states更新されるパーティション。
partition_indexパーティションのインデックス。
controller_epochコントローラー epoch。
leader現在のパーティションリーダーであるブローカーの ID。
leader_epochこのパーティションのリーダー エポック。
isrこのパーティションの ISR にあるブローカー。
zk_versionZookeeper のバージョン。
replicasこのパーティションの全てのレプリカ。
offline_replicasオフラインのこのパーティションのレプリカ。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド
live_brokers
idブローカーid。
endpointsブローカーのエンドポイント。The broker endpoints.
portこのエンドポイントのポート。
hostこのエンドポイントのホスト名。
listenerリスナー名。
security_protocolセキュリティ プロトコルの型
_tagged_fieldsタグ付きのフィールド
rackこのブローカーが所属するラック。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

Responses:

UpdateMetadata Response (Version: 0) => error_code 
  error_code => INT16
フィールド 説明
error_codeエラーコード。あるいはエラーが無かった場合は0。

UpdateMetadata Response (Version: 1) => error_code 
  error_code => INT16
フィールド 説明
error_codeエラーコード。あるいはエラーが無かった場合は0。

UpdateMetadata Response (Version: 2) => error_code 
  error_code => INT16
フィールド 説明
error_codeエラーコード。あるいはエラーが無かった場合は0。

UpdateMetadata Response (Version: 3) => error_code 
  error_code => INT16
フィールド 説明
error_codeエラーコード。あるいはエラーが無かった場合は0。

UpdateMetadata Response (Version: 4) => error_code 
  error_code => INT16
フィールド 説明
error_codeエラーコード。あるいはエラーが無かった場合は0。

UpdateMetadata Response (Version: 5) => error_code 
  error_code => INT16
フィールド 説明
error_codeエラーコード。あるいはエラーが無かった場合は0。

UpdateMetadata Response (Version: 6) => error_code TAG_BUFFER 
  error_code => INT16
フィールド 説明
error_codeエラーコード。あるいはエラーが無かった場合は0。
_tagged_fieldsタグ付きのフィールド

ControlledShutdown API (Key: 7):
Requests:

ControlledShutdown Request (Version: 0) => broker_id 
  broker_id => INT32
フィールド 説明
broker_id制御されたシャットダウンがリクエストされたブローカーのid。

ControlledShutdown Request (Version: 1) => broker_id 
  broker_id => INT32
フィールド 説明
broker_id制御されたシャットダウンがリクエストされたブローカーのid。

ControlledShutdown Request (Version: 2) => broker_id broker_epoch 
  broker_id => INT32
  broker_epoch => INT64
フィールド 説明
broker_id制御されたシャットダウンがリクエストされたブローカーのid。
broker_epochブローカーのエポック。

ControlledShutdown Request (Version: 3) => broker_id broker_epoch TAG_BUFFER 
  broker_id => INT32
  broker_epoch => INT64
フィールド 説明
broker_id制御されたシャットダウンがリクエストされたブローカーのid。
broker_epochブローカーのエポック。
_tagged_fieldsタグ付きのフィールド

Responses:

ControlledShutdown Response (Version: 0) => error_code [remaining_partitions] 
  error_code => INT16
  remaining_partitions => topic_name partition_index 
    topic_name => STRING
    partition_index => INT32
フィールド 説明
error_codeトップレベルのエラーコード。
remaining_partitionsブローカーがまだ先導するパーティション。
topic_nameトピックの名前。
partition_indexパーティションのインデックス。

ControlledShutdown Response (Version: 1) => error_code [remaining_partitions] 
  error_code => INT16
  remaining_partitions => topic_name partition_index 
    topic_name => STRING
    partition_index => INT32
フィールド 説明
error_codeトップレベルのエラーコード。
remaining_partitionsブローカーがまだ先導するパーティション。
topic_nameトピックの名前。
partition_indexパーティションのインデックス。

ControlledShutdown Response (Version: 2) => error_code [remaining_partitions] 
  error_code => INT16
  remaining_partitions => topic_name partition_index 
    topic_name => STRING
    partition_index => INT32
フィールド 説明
error_codeトップレベルのエラーコード。
remaining_partitionsブローカーがまだ先導するパーティション。
topic_nameトピックの名前。
partition_indexパーティションのインデックス。

ControlledShutdown Response (Version: 3) => error_code [remaining_partitions] TAG_BUFFER 
  error_code => INT16
  remaining_partitions => topic_name partition_index TAG_BUFFER 
    topic_name => COMPACT_STRING
    partition_index => INT32
フィールド 説明
error_codeトップレベルのエラーコード。
remaining_partitionsブローカーがまだ先導するパーティション。
topic_nameトピックの名前。
partition_indexパーティションのインデックス。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

OffsetCommit API (Key: 8):
Requests:

OffsetCommit Request (Version: 0) => group_id [topics] 
  group_id => STRING
  topics => name [partitions] 
    name => STRING
    partitions => partition_index committed_offset committed_metadata 
      partition_index => INT32
      committed_offset => INT64
      committed_metadata => NULLABLE_STRING
フィールド 説明
group_idユニークなグループの識別子。
topicsオフセットをコミットするトピック。
名前トピック名。
partitionsオフセットをコミットする各トピック。
partition_indexパーティションのインデックス。
committed_offsetコミットされるメッセージのオフセット。
committed_metadataクライアントが維持したい関連メタデータ。

OffsetCommit Request (Version: 1) => group_id generation_id member_id [topics] 
  group_id => STRING
  generation_id => INT32
  member_id => STRING
  topics => name [partitions] 
    name => STRING
    partitions => partition_index committed_offset commit_timestamp committed_metadata 
      partition_index => INT32
      committed_offset => INT64
      commit_timestamp => INT64
      committed_metadata => NULLABLE_STRING
フィールド 説明
group_idユニークなグループの識別子。
generation_idグループの世代。
member_idグループ コーディネータによって割り当てられたメンバーID。
topicsオフセットをコミットするトピック。
名前トピック名。
partitionsオフセットをコミットする各トピック。
partition_indexパーティションのインデックス。
committed_offsetコミットされるメッセージのオフセット。
commit_timestampコミットのタイムスタンプ。
committed_metadataクライアントが維持したい関連メタデータ。

OffsetCommit Request (Version: 2) => group_id generation_id member_id retention_time_ms [topics] 
  group_id => STRING
  generation_id => INT32
  member_id => STRING
  retention_time_ms => INT64
  topics => name [partitions] 
    name => STRING
    partitions => partition_index committed_offset committed_metadata 
      partition_index => INT32
      committed_offset => INT64
      committed_metadata => NULLABLE_STRING
フィールド 説明
group_idユニークなグループの識別子。
generation_idグループの世代。
member_idグループ コーディネータによって割り当てられたメンバーID。
retention_time_msオフセットを維持する時間のミリ秒。
topicsオフセットをコミットするトピック。
名前トピック名。
partitionsオフセットをコミットする各トピック。
partition_indexパーティションのインデックス。
committed_offsetコミットされるメッセージのオフセット。
committed_metadataクライアントが維持したい関連メタデータ。

OffsetCommit Request (Version: 3) => group_id generation_id member_id retention_time_ms [topics] 
  group_id => STRING
  generation_id => INT32
  member_id => STRING
  retention_time_ms => INT64
  topics => name [partitions] 
    name => STRING
    partitions => partition_index committed_offset committed_metadata 
      partition_index => INT32
      committed_offset => INT64
      committed_metadata => NULLABLE_STRING
フィールド 説明
group_idユニークなグループの識別子。
generation_idグループの世代。
member_idグループ コーディネータによって割り当てられたメンバーID。
retention_time_msオフセットを維持する時間のミリ秒。
topicsオフセットをコミットするトピック。
名前トピック名。
partitionsオフセットをコミットする各トピック。
partition_indexパーティションのインデックス。
committed_offsetコミットされるメッセージのオフセット。
committed_metadataクライアントが維持したい関連メタデータ。

OffsetCommit Request (Version: 4) => group_id generation_id member_id retention_time_ms [topics] 
  group_id => STRING
  generation_id => INT32
  member_id => STRING
  retention_time_ms => INT64
  topics => name [partitions] 
    name => STRING
    partitions => partition_index committed_offset committed_metadata 
      partition_index => INT32
      committed_offset => INT64
      committed_metadata => NULLABLE_STRING
フィールド 説明
group_idユニークなグループの識別子。
generation_idグループの世代。
member_idグループ コーディネータによって割り当てられたメンバーID。
retention_time_msオフセットを維持する時間のミリ秒。
topicsオフセットをコミットするトピック。
名前トピック名。
partitionsオフセットをコミットする各トピック。
partition_indexパーティションのインデックス。
committed_offsetコミットされるメッセージのオフセット。
committed_metadataクライアントが維持したい関連メタデータ。

OffsetCommit Request (Version: 5) => group_id generation_id member_id [topics] 
  group_id => STRING
  generation_id => INT32
  member_id => STRING
  topics => name [partitions] 
    name => STRING
    partitions => partition_index committed_offset committed_metadata 
      partition_index => INT32
      committed_offset => INT64
      committed_metadata => NULLABLE_STRING
フィールド 説明
group_idユニークなグループの識別子。
generation_idグループの世代。
member_idグループ コーディネータによって割り当てられたメンバーID。
topicsオフセットをコミットするトピック。
名前トピック名。
partitionsオフセットをコミットする各トピック。
partition_indexパーティションのインデックス。
committed_offsetコミットされるメッセージのオフセット。
committed_metadataクライアントが維持したい関連メタデータ。

OffsetCommit Request (Version: 6) => group_id generation_id member_id [topics] 
  group_id => STRING
  generation_id => INT32
  member_id => STRING
  topics => name [partitions] 
    name => STRING
    partitions => partition_index committed_offset committed_leader_epoch committed_metadata 
      partition_index => INT32
      committed_offset => INT64
      committed_leader_epoch => INT32
      committed_metadata => NULLABLE_STRING
フィールド 説明
group_idユニークなグループの識別子。
generation_idグループの世代。
member_idグループ コーディネータによって割り当てられたメンバーID。
topicsオフセットをコミットするトピック。
名前トピック名。
partitionsオフセットをコミットする各トピック。
partition_indexパーティションのインデックス。
committed_offsetコミットされるメッセージのオフセット。
committed_leader_epochこのパーティションのリーダー エポック。
committed_metadataクライアントが維持したい関連メタデータ。

OffsetCommit Request (Version: 7) => group_id generation_id member_id group_instance_id [topics] 
  group_id => STRING
  generation_id => INT32
  member_id => STRING
  group_instance_id => NULLABLE_STRING
  topics => name [partitions] 
    name => STRING
    partitions => partition_index committed_offset committed_leader_epoch committed_metadata 
      partition_index => INT32
      committed_offset => INT64
      committed_leader_epoch => INT32
      committed_metadata => NULLABLE_STRING
フィールド 説明
group_idユニークなグループの識別子。
generation_idグループの世代。
member_idグループ コーディネータによって割り当てられたメンバーID。
group_instance_idエンドユーザによって提供されるコンシューマ インスタンスの一意の識別子。
topicsオフセットをコミットするトピック。
名前トピック名。
partitionsオフセットをコミットする各トピック。
partition_indexパーティションのインデックス。
committed_offsetコミットされるメッセージのオフセット。
committed_leader_epochこのパーティションのリーダー エポック。
committed_metadataクライアントが維持したい関連メタデータ。

OffsetCommit Request (Version: 8) => group_id generation_id member_id group_instance_id [topics] TAG_BUFFER 
  group_id => COMPACT_STRING
  generation_id => INT32
  member_id => COMPACT_STRING
  group_instance_id => COMPACT_NULLABLE_STRING
  topics => name [partitions] TAG_BUFFER 
    name => COMPACT_STRING
    partitions => partition_index committed_offset committed_leader_epoch committed_metadata TAG_BUFFER 
      partition_index => INT32
      committed_offset => INT64
      committed_leader_epoch => INT32
      committed_metadata => COMPACT_NULLABLE_STRING
フィールド 説明
group_idユニークなグループの識別子。
generation_idグループの世代。
member_idグループ コーディネータによって割り当てられたメンバーID。
group_instance_idエンドユーザによって提供されるコンシューマ インスタンスの一意の識別子。
topicsオフセットをコミットするトピック。
名前トピック名。
partitionsオフセットをコミットする各トピック。
partition_indexパーティションのインデックス。
committed_offsetコミットされるメッセージのオフセット。
committed_leader_epochこのパーティションのリーダー エポック。
committed_metadataクライアントが維持したい関連メタデータ。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

Responses:

OffsetCommit Response (Version: 0) => [topics] 
  topics => name [partitions] 
    name => STRING
    partitions => partition_index error_code 
      partition_index => INT32
      error_code => INT16
フィールド 説明
topics各トピックについての応答。
名前トピック名。
partitionsトピック内の各パーティションについての応答。
partition_indexパーティションのインデックス。
error_codeエラーコード。あるいはエラーが無かった場合は0。

OffsetCommit Response (Version: 1) => [topics] 
  topics => name [partitions] 
    name => STRING
    partitions => partition_index error_code 
      partition_index => INT32
      error_code => INT16
フィールド 説明
topics各トピックについての応答。
名前トピック名。
partitionsトピック内の各パーティションについての応答。
partition_indexパーティションのインデックス。
error_codeエラーコード。あるいはエラーが無かった場合は0。

OffsetCommit Response (Version: 2) => [topics] 
  topics => name [partitions] 
    name => STRING
    partitions => partition_index error_code 
      partition_index => INT32
      error_code => INT16
フィールド 説明
topics各トピックについての応答。
名前トピック名。
partitionsトピック内の各パーティションについての応答。
partition_indexパーティションのインデックス。
error_codeエラーコード。あるいはエラーが無かった場合は0。

OffsetCommit Response (Version: 3) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index error_code 
      partition_index => INT32
      error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
topics各トピックについての応答。
名前トピック名。
partitionsトピック内の各パーティションについての応答。
partition_indexパーティションのインデックス。
error_codeエラーコード。あるいはエラーが無かった場合は0。

OffsetCommit Response (Version: 4) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index error_code 
      partition_index => INT32
      error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
topics各トピックについての応答。
名前トピック名。
partitionsトピック内の各パーティションについての応答。
partition_indexパーティションのインデックス。
error_codeエラーコード。あるいはエラーが無かった場合は0。

OffsetCommit Response (Version: 5) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index error_code 
      partition_index => INT32
      error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
topics各トピックについての応答。
名前トピック名。
partitionsトピック内の各パーティションについての応答。
partition_indexパーティションのインデックス。
error_codeエラーコード。あるいはエラーが無かった場合は0。

OffsetCommit Response (Version: 6) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index error_code 
      partition_index => INT32
      error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
topics各トピックについての応答。
名前トピック名。
partitionsトピック内の各パーティションについての応答。
partition_indexパーティションのインデックス。
error_codeエラーコード。あるいはエラーが無かった場合は0。

OffsetCommit Response (Version: 7) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index error_code 
      partition_index => INT32
      error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
topics各トピックについての応答。
名前トピック名。
partitionsトピック内の各パーティションについての応答。
partition_indexパーティションのインデックス。
error_codeエラーコード。あるいはエラーが無かった場合は0。

OffsetCommit Response (Version: 8) => throttle_time_ms [topics] TAG_BUFFER 
  throttle_time_ms => INT32
  topics => name [partitions] TAG_BUFFER 
    name => COMPACT_STRING
    partitions => partition_index error_code TAG_BUFFER 
      partition_index => INT32
      error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
topics各トピックについての応答。
名前トピック名。
partitionsトピック内の各パーティションについての応答。
partition_indexパーティションのインデックス。
error_codeエラーコード。あるいはエラーが無かった場合は0。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

OffsetFetch API (Key: 9):
Requests:

OffsetFetch Request (Version: 0) => group_id [topics] 
  group_id => STRING
  topics => name [partition_indexes] 
    name => STRING
    partition_indexes => INT32
フィールド 説明
group_idオフセットをフェッチするグループ。
topicsオフセットをフェッチする各トピック。または全てのトピックのオフセットをフェッチする場合は null。
名前トピック名。
partition_indexesオフセットをフェッチするパーティションインデックス。

OffsetFetch Request (Version: 1) => group_id [topics] 
  group_id => STRING
  topics => name [partition_indexes] 
    name => STRING
    partition_indexes => INT32
フィールド 説明
group_idオフセットをフェッチするグループ。
topicsオフセットをフェッチする各トピック。または全てのトピックのオフセットをフェッチする場合は null。
名前トピック名。
partition_indexesオフセットをフェッチするパーティションインデックス。

OffsetFetch Request (Version: 2) => group_id [topics] 
  group_id => STRING
  topics => name [partition_indexes] 
    name => STRING
    partition_indexes => INT32
フィールド 説明
group_idオフセットをフェッチするグループ。
topicsオフセットをフェッチする各トピック。または全てのトピックのオフセットをフェッチする場合は null。
名前トピック名。
partition_indexesオフセットをフェッチするパーティションインデックス。

OffsetFetch Request (Version: 3) => group_id [topics] 
  group_id => STRING
  topics => name [partition_indexes] 
    name => STRING
    partition_indexes => INT32
フィールド 説明
group_idオフセットをフェッチするグループ。
topicsオフセットをフェッチする各トピック。または全てのトピックのオフセットをフェッチする場合は null。
名前トピック名。
partition_indexesオフセットをフェッチするパーティションインデックス。

OffsetFetch Request (Version: 4) => group_id [topics] 
  group_id => STRING
  topics => name [partition_indexes] 
    name => STRING
    partition_indexes => INT32
フィールド 説明
group_idオフセットをフェッチするグループ。
topicsオフセットをフェッチする各トピック。または全てのトピックのオフセットをフェッチする場合は null。
名前トピック名。
partition_indexesオフセットをフェッチするパーティションインデックス。

OffsetFetch Request (Version: 5) => group_id [topics] 
  group_id => STRING
  topics => name [partition_indexes] 
    name => STRING
    partition_indexes => INT32
フィールド 説明
group_idオフセットをフェッチするグループ。
topicsオフセットをフェッチする各トピック。または全てのトピックのオフセットをフェッチする場合は null。
名前トピック名。
partition_indexesオフセットをフェッチするパーティションインデックス。

OffsetFetch Request (Version: 6) => group_id [topics] TAG_BUFFER 
  group_id => COMPACT_STRING
  topics => name [partition_indexes] TAG_BUFFER 
    name => COMPACT_STRING
    partition_indexes => INT32
フィールド 説明
group_idオフセットをフェッチするグループ。
topicsオフセットをフェッチする各トピック。または全てのトピックのオフセットをフェッチする場合は null。
名前トピック名。
partition_indexesオフセットをフェッチするパーティションインデックス。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

OffsetFetch Request (Version: 7) => group_id [topics] require_stable TAG_BUFFER 
  group_id => COMPACT_STRING
  topics => name [partition_indexes] TAG_BUFFER 
    name => COMPACT_STRING
    partition_indexes => INT32
  require_stable => BOOLEAN
フィールド 説明
group_idオフセットをフェッチするグループ。
topicsオフセットをフェッチする各トピック。または全てのトピックのオフセットをフェッチする場合は null。
名前トピック名。
partition_indexesオフセットをフェッチするパーティションインデックス。
_tagged_fieldsタグ付きのフィールド
require_stableブローカーが不安定なオフセットを返すことを保留する必要があるが、パーティションに再実行可能なエラーコードを設定するかどうか。
_tagged_fieldsタグ付きのフィールド

Responses:

OffsetFetch Response (Version: 0) => [topics] 
  topics => name [partitions] 
    name => STRING
    partitions => partition_index committed_offset metadata error_code 
      partition_index => INT32
      committed_offset => INT64
      metadata => NULLABLE_STRING
      error_code => INT16
フィールド 説明
topicsトピックごとの応答。
名前トピック名。
partitionsパーティションごとの応答
partition_indexパーティションのインデックス。
committed_offsetコミットされたメッセージのオフセット。
metadataパーティションのメタデータ。
error_codeエラーコード。あるいはエラーが無かった場合は0。

OffsetFetch Response (Version: 1) => [topics] 
  topics => name [partitions] 
    name => STRING
    partitions => partition_index committed_offset metadata error_code 
      partition_index => INT32
      committed_offset => INT64
      metadata => NULLABLE_STRING
      error_code => INT16
フィールド 説明
topicsトピックごとの応答。
名前トピック名。
partitionsパーティションごとの応答
partition_indexパーティションのインデックス。
committed_offsetコミットされたメッセージのオフセット。
metadataパーティションのメタデータ。
error_codeエラーコード。あるいはエラーが無かった場合は0。

OffsetFetch Response (Version: 2) => [topics] error_code 
  topics => name [partitions] 
    name => STRING
    partitions => partition_index committed_offset metadata error_code 
      partition_index => INT32
      committed_offset => INT64
      metadata => NULLABLE_STRING
      error_code => INT16
  error_code => INT16
フィールド 説明
topicsトピックごとの応答。
名前トピック名。
partitionsパーティションごとの応答
partition_indexパーティションのインデックス。
committed_offsetコミットされたメッセージのオフセット。
metadataパーティションのメタデータ。
error_codeエラーコード。あるいはエラーが無かった場合は0。
error_codeトップレベルのエラーコード。あるいはエラーが無い場合は 0。

OffsetFetch Response (Version: 3) => throttle_time_ms [topics] error_code 
  throttle_time_ms => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index committed_offset metadata error_code 
      partition_index => INT32
      committed_offset => INT64
      metadata => NULLABLE_STRING
      error_code => INT16
  error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
topicsトピックごとの応答。
名前トピック名。
partitionsパーティションごとの応答
partition_indexパーティションのインデックス。
committed_offsetコミットされたメッセージのオフセット。
metadataパーティションのメタデータ。
error_codeエラーコード。あるいはエラーが無かった場合は0。
error_codeトップレベルのエラーコード。あるいはエラーが無い場合は 0。

OffsetFetch Response (Version: 4) => throttle_time_ms [topics] error_code 
  throttle_time_ms => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index committed_offset metadata error_code 
      partition_index => INT32
      committed_offset => INT64
      metadata => NULLABLE_STRING
      error_code => INT16
  error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
topicsトピックごとの応答。
名前トピック名。
partitionsパーティションごとの応答
partition_indexパーティションのインデックス。
committed_offsetコミットされたメッセージのオフセット。
metadataパーティションのメタデータ。
error_codeエラーコード。あるいはエラーが無かった場合は0。
error_codeトップレベルのエラーコード。あるいはエラーが無い場合は 0。

OffsetFetch Response (Version: 5) => throttle_time_ms [topics] error_code 
  throttle_time_ms => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index committed_offset committed_leader_epoch metadata error_code 
      partition_index => INT32
      committed_offset => INT64
      committed_leader_epoch => INT32
      metadata => NULLABLE_STRING
      error_code => INT16
  error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
topicsトピックごとの応答。
名前トピック名。
partitionsパーティションごとの応答
partition_indexパーティションのインデックス。
committed_offsetコミットされたメッセージのオフセット。
committed_leader_epochリーダーのepoch。
metadataパーティションのメタデータ。
error_codeエラーコード。あるいはエラーが無かった場合は0。
error_codeトップレベルのエラーコード。あるいはエラーが無い場合は 0。

OffsetFetch Response (Version: 6) => throttle_time_ms [topics] error_code TAG_BUFFER 
  throttle_time_ms => INT32
  topics => name [partitions] TAG_BUFFER 
    name => COMPACT_STRING
    partitions => partition_index committed_offset committed_leader_epoch metadata error_code TAG_BUFFER 
      partition_index => INT32
      committed_offset => INT64
      committed_leader_epoch => INT32
      metadata => COMPACT_NULLABLE_STRING
      error_code => INT16
  error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
topicsトピックごとの応答。
名前トピック名。
partitionsパーティションごとの応答
partition_indexパーティションのインデックス。
committed_offsetコミットされたメッセージのオフセット。
committed_leader_epochリーダーのepoch。
metadataパーティションのメタデータ。
error_codeエラーコード。あるいはエラーが無かった場合は0。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド
error_codeトップレベルのエラーコード。あるいはエラーが無い場合は 0。
_tagged_fieldsタグ付きのフィールド

OffsetFetch Response (Version: 7) => throttle_time_ms [topics] error_code TAG_BUFFER 
  throttle_time_ms => INT32
  topics => name [partitions] TAG_BUFFER 
    name => COMPACT_STRING
    partitions => partition_index committed_offset committed_leader_epoch metadata error_code TAG_BUFFER 
      partition_index => INT32
      committed_offset => INT64
      committed_leader_epoch => INT32
      metadata => COMPACT_NULLABLE_STRING
      error_code => INT16
  error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
topicsトピックごとの応答。
名前トピック名。
partitionsパーティションごとの応答
partition_indexパーティションのインデックス。
committed_offsetコミットされたメッセージのオフセット。
committed_leader_epochリーダーのepoch。
metadataパーティションのメタデータ。
error_codeエラーコード。あるいはエラーが無かった場合は0。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド
error_codeトップレベルのエラーコード。あるいはエラーが無い場合は 0。
_tagged_fieldsタグ付きのフィールド

FindCoordinator API (Key: 10):
Requests:

FindCoordinator Request (Version: 0) => key 
  key => STRING
フィールド 説明
キーコーディネータ キー。

FindCoordinator Request (Version: 1) => key key_type 
  key => STRING
  key_type => INT8
フィールド 説明
キーコーディネータ キー。
key_typeコーディネータ キーの型。(グループ、トランザクションなど)

FindCoordinator Request (Version: 2) => key key_type 
  key => STRING
  key_type => INT8
フィールド 説明
キーコーディネータ キー。
key_typeコーディネータ キーの型。(グループ、トランザクションなど)

FindCoordinator Request (Version: 3) => key key_type TAG_BUFFER 
  key => COMPACT_STRING
  key_type => INT8
フィールド 説明
キーコーディネータ キー。
key_typeコーディネータ キーの型。(グループ、トランザクションなど)
_tagged_fieldsタグ付きのフィールド

Responses:

FindCoordinator Response (Version: 0) => error_code node_id host port 
  error_code => INT16
  node_id => INT32
  host => STRING
  port => INT32
フィールド 説明
error_codeエラーコード。あるいはエラーが無かった場合は0。
node_idノードid。
hostホスト名。
portポート。

FindCoordinator Response (Version: 1) => throttle_time_ms error_code error_message node_id host port 
  throttle_time_ms => INT32
  error_code => INT16
  error_message => NULLABLE_STRING
  node_id => INT32
  host => STRING
  port => INT32
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。
error_messageエラーメッセージ。あるいはエラーが無かった場合はnull。
node_idノードid。
hostホスト名。
portポート。

FindCoordinator Response (Version: 2) => throttle_time_ms error_code error_message node_id host port 
  throttle_time_ms => INT32
  error_code => INT16
  error_message => NULLABLE_STRING
  node_id => INT32
  host => STRING
  port => INT32
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。
error_messageエラーメッセージ。あるいはエラーが無かった場合はnull。
node_idノードid。
hostホスト名。
portポート。

FindCoordinator Response (Version: 3) => throttle_time_ms error_code error_message node_id host port TAG_BUFFER 
  throttle_time_ms => INT32
  error_code => INT16
  error_message => COMPACT_NULLABLE_STRING
  node_id => INT32
  host => COMPACT_STRING
  port => INT32
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。
error_messageエラーメッセージ。あるいはエラーが無かった場合はnull。
node_idノードid。
hostホスト名。
portポート。
_tagged_fieldsタグ付きのフィールド

JoinGroup API (Key: 11):
Requests:

JoinGroup Request (Version: 0) => group_id session_timeout_ms member_id protocol_type [protocols] 
  group_id => STRING
  session_timeout_ms => INT32
  member_id => STRING
  protocol_type => STRING
  protocols => name metadata 
    name => STRING
    metadata => BYTES
フィールド 説明
group_idグループ識別子。
session_timeout_msもしコーディネータがこのタイムアウトのミリ秒の後でハートビートを受け取らない場合は、コンシューマが死亡したと見なします。
member_idグループ コーディネータによって割り当てられたメンバーid。
protocol_type参加するグループによって実装されるプロトコルのクラスの一意の名前。
protocolsメンバーがサポートするプロトコルのリスト。
名前プロトコル名。
metadataプロトコルのメタデータ。

JoinGroup Request (Version: 1) => group_id session_timeout_ms rebalance_timeout_ms member_id protocol_type [protocols] 
  group_id => STRING
  session_timeout_ms => INT32
  rebalance_timeout_ms => INT32
  member_id => STRING
  protocol_type => STRING
  protocols => name metadata 
    name => STRING
    metadata => BYTES
フィールド 説明
group_idグループ識別子。
session_timeout_msもしコーディネータがこのタイムアウトのミリ秒の後でハートビートを受け取らない場合は、コンシューマが死亡したと見なします。
rebalance_timeout_msグループのリバランス時にコーディネータが各メンバーが再入会するのを待つだろう最大時間のミリ秒。
member_idグループ コーディネータによって割り当てられたメンバーid。
protocol_type参加するグループによって実装されるプロトコルのクラスの一意の名前。
protocolsメンバーがサポートするプロトコルのリスト。
名前プロトコル名。
metadataプロトコルのメタデータ。

JoinGroup Request (Version: 2) => group_id session_timeout_ms rebalance_timeout_ms member_id protocol_type [protocols] 
  group_id => STRING
  session_timeout_ms => INT32
  rebalance_timeout_ms => INT32
  member_id => STRING
  protocol_type => STRING
  protocols => name metadata 
    name => STRING
    metadata => BYTES
フィールド 説明
group_idグループ識別子。
session_timeout_msもしコーディネータがこのタイムアウトのミリ秒の後でハートビートを受け取らない場合は、コンシューマが死亡したと見なします。
rebalance_timeout_msグループのリバランス時にコーディネータが各メンバーが再入会するのを待つだろう最大時間のミリ秒。
member_idグループ コーディネータによって割り当てられたメンバーid。
protocol_type参加するグループによって実装されるプロトコルのクラスの一意の名前。
protocolsメンバーがサポートするプロトコルのリスト。
名前プロトコル名。
metadataプロトコルのメタデータ。

JoinGroup Request (Version: 3) => group_id session_timeout_ms rebalance_timeout_ms member_id protocol_type [protocols] 
  group_id => STRING
  session_timeout_ms => INT32
  rebalance_timeout_ms => INT32
  member_id => STRING
  protocol_type => STRING
  protocols => name metadata 
    name => STRING
    metadata => BYTES
フィールド 説明
group_idグループ識別子。
session_timeout_msもしコーディネータがこのタイムアウトのミリ秒の後でハートビートを受け取らない場合は、コンシューマが死亡したと見なします。
rebalance_timeout_msグループのリバランス時にコーディネータが各メンバーが再入会するのを待つだろう最大時間のミリ秒。
member_idグループ コーディネータによって割り当てられたメンバーid。
protocol_type参加するグループによって実装されるプロトコルのクラスの一意の名前。
protocolsメンバーがサポートするプロトコルのリスト。
名前プロトコル名。
metadataプロトコルのメタデータ。

JoinGroup Request (Version: 4) => group_id session_timeout_ms rebalance_timeout_ms member_id protocol_type [protocols] 
  group_id => STRING
  session_timeout_ms => INT32
  rebalance_timeout_ms => INT32
  member_id => STRING
  protocol_type => STRING
  protocols => name metadata 
    name => STRING
    metadata => BYTES
フィールド 説明
group_idグループ識別子。
session_timeout_msもしコーディネータがこのタイムアウトのミリ秒の後でハートビートを受け取らない場合は、コンシューマが死亡したと見なします。
rebalance_timeout_msグループのリバランス時にコーディネータが各メンバーが再入会するのを待つだろう最大時間のミリ秒。
member_idグループ コーディネータによって割り当てられたメンバーid。
protocol_type参加するグループによって実装されるプロトコルのクラスの一意の名前。
protocolsメンバーがサポートするプロトコルのリスト。
名前プロトコル名。
metadataプロトコルのメタデータ。

JoinGroup Request (Version: 5) => group_id session_timeout_ms rebalance_timeout_ms member_id group_instance_id protocol_type [protocols] 
  group_id => STRING
  session_timeout_ms => INT32
  rebalance_timeout_ms => INT32
  member_id => STRING
  group_instance_id => NULLABLE_STRING
  protocol_type => STRING
  protocols => name metadata 
    name => STRING
    metadata => BYTES
フィールド 説明
group_idグループ識別子。
session_timeout_msもしコーディネータがこのタイムアウトのミリ秒の後でハートビートを受け取らない場合は、コンシューマが死亡したと見なします。
rebalance_timeout_msグループのリバランス時にコーディネータが各メンバーが再入会するのを待つだろう最大時間のミリ秒。
member_idグループ コーディネータによって割り当てられたメンバーid。
group_instance_idエンドユーザによって提供されるコンシューマ インスタンスの一意の識別子。
protocol_type参加するグループによって実装されるプロトコルのクラスの一意の名前。
protocolsメンバーがサポートするプロトコルのリスト。
名前プロトコル名。
metadataプロトコルのメタデータ。

JoinGroup Request (Version: 6) => group_id session_timeout_ms rebalance_timeout_ms member_id group_instance_id protocol_type [protocols] TAG_BUFFER 
  group_id => COMPACT_STRING
  session_timeout_ms => INT32
  rebalance_timeout_ms => INT32
  member_id => COMPACT_STRING
  group_instance_id => COMPACT_NULLABLE_STRING
  protocol_type => COMPACT_STRING
  protocols => name metadata TAG_BUFFER 
    name => COMPACT_STRING
    metadata => COMPACT_BYTES
フィールド 説明
group_idグループ識別子。
session_timeout_msもしコーディネータがこのタイムアウトのミリ秒の後でハートビートを受け取らない場合は、コンシューマが死亡したと見なします。
rebalance_timeout_msグループのリバランス時にコーディネータが各メンバーが再入会するのを待つだろう最大時間のミリ秒。
member_idグループ コーディネータによって割り当てられたメンバーid。
group_instance_idエンドユーザによって提供されるコンシューマ インスタンスの一意の識別子。
protocol_type参加するグループによって実装されるプロトコルのクラスの一意の名前。
protocolsメンバーがサポートするプロトコルのリスト。
名前プロトコル名。
metadataプロトコルのメタデータ。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

JoinGroup Request (Version: 7) => group_id session_timeout_ms rebalance_timeout_ms member_id group_instance_id protocol_type [protocols] TAG_BUFFER 
  group_id => COMPACT_STRING
  session_timeout_ms => INT32
  rebalance_timeout_ms => INT32
  member_id => COMPACT_STRING
  group_instance_id => COMPACT_NULLABLE_STRING
  protocol_type => COMPACT_STRING
  protocols => name metadata TAG_BUFFER 
    name => COMPACT_STRING
    metadata => COMPACT_BYTES
フィールド 説明
group_idグループ識別子。
session_timeout_msもしコーディネータがこのタイムアウトのミリ秒の後でハートビートを受け取らない場合は、コンシューマが死亡したと見なします。
rebalance_timeout_msグループのリバランス時にコーディネータが各メンバーが再入会するのを待つだろう最大時間のミリ秒。
member_idグループ コーディネータによって割り当てられたメンバーid。
group_instance_idエンドユーザによって提供されるコンシューマ インスタンスの一意の識別子。
protocol_type参加するグループによって実装されるプロトコルのクラスの一意の名前。
protocolsメンバーがサポートするプロトコルのリスト。
名前プロトコル名。
metadataプロトコルのメタデータ。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

Responses:

JoinGroup Response (Version: 0) => error_code generation_id protocol_name leader member_id [members] 
  error_code => INT16
  generation_id => INT32
  protocol_name => STRING
  leader => STRING
  member_id => STRING
  members => member_id metadata 
    member_id => STRING
    metadata => BYTES
フィールド 説明
error_codeエラーコード。あるいはエラーが無かった場合は0。
generation_idグループの生成ID。
protocol_nameコーディネータによって選択されたグループ プロトコル。
leaderグループのリーダー。
member_idグループ コーディネータによって割り当てられたメンバーID。
members
member_idグループ メンバー ID。
metadataグループ メンバーのメタデータ。

JoinGroup Response (Version: 1) => error_code generation_id protocol_name leader member_id [members] 
  error_code => INT16
  generation_id => INT32
  protocol_name => STRING
  leader => STRING
  member_id => STRING
  members => member_id metadata 
    member_id => STRING
    metadata => BYTES
フィールド 説明
error_codeエラーコード。あるいはエラーが無かった場合は0。
generation_idグループの生成ID。
protocol_nameコーディネータによって選択されたグループ プロトコル。
leaderグループのリーダー。
member_idグループ コーディネータによって割り当てられたメンバーID。
members
member_idグループ メンバー ID。
metadataグループ メンバーのメタデータ。

JoinGroup Response (Version: 2) => throttle_time_ms error_code generation_id protocol_name leader member_id [members] 
  throttle_time_ms => INT32
  error_code => INT16
  generation_id => INT32
  protocol_name => STRING
  leader => STRING
  member_id => STRING
  members => member_id metadata 
    member_id => STRING
    metadata => BYTES
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。
generation_idグループの生成ID。
protocol_nameコーディネータによって選択されたグループ プロトコル。
leaderグループのリーダー。
member_idグループ コーディネータによって割り当てられたメンバーID。
members
member_idグループ メンバー ID。
metadataグループ メンバーのメタデータ。

JoinGroup Response (Version: 3) => throttle_time_ms error_code generation_id protocol_name leader member_id [members] 
  throttle_time_ms => INT32
  error_code => INT16
  generation_id => INT32
  protocol_name => STRING
  leader => STRING
  member_id => STRING
  members => member_id metadata 
    member_id => STRING
    metadata => BYTES
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。
generation_idグループの生成ID。
protocol_nameコーディネータによって選択されたグループ プロトコル。
leaderグループのリーダー。
member_idグループ コーディネータによって割り当てられたメンバーID。
members
member_idグループ メンバー ID。
metadataグループ メンバーのメタデータ。

JoinGroup Response (Version: 4) => throttle_time_ms error_code generation_id protocol_name leader member_id [members] 
  throttle_time_ms => INT32
  error_code => INT16
  generation_id => INT32
  protocol_name => STRING
  leader => STRING
  member_id => STRING
  members => member_id metadata 
    member_id => STRING
    metadata => BYTES
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。
generation_idグループの生成ID。
protocol_nameコーディネータによって選択されたグループ プロトコル。
leaderグループのリーダー。
member_idグループ コーディネータによって割り当てられたメンバーID。
members
member_idグループ メンバー ID。
metadataグループ メンバーのメタデータ。

JoinGroup Response (Version: 5) => throttle_time_ms error_code generation_id protocol_name leader member_id [members] 
  throttle_time_ms => INT32
  error_code => INT16
  generation_id => INT32
  protocol_name => STRING
  leader => STRING
  member_id => STRING
  members => member_id group_instance_id metadata 
    member_id => STRING
    group_instance_id => NULLABLE_STRING
    metadata => BYTES
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。
generation_idグループの生成ID。
protocol_nameコーディネータによって選択されたグループ プロトコル。
leaderグループのリーダー。
member_idグループ コーディネータによって割り当てられたメンバーID。
members
member_idグループ メンバー ID。
group_instance_idエンドユーザによって提供されるコンシューマ インスタンスの一意の識別子。
metadataグループ メンバーのメタデータ。

JoinGroup Response (Version: 6) => throttle_time_ms error_code generation_id protocol_name leader member_id [members] TAG_BUFFER 
  throttle_time_ms => INT32
  error_code => INT16
  generation_id => INT32
  protocol_name => COMPACT_STRING
  leader => COMPACT_STRING
  member_id => COMPACT_STRING
  members => member_id group_instance_id metadata TAG_BUFFER 
    member_id => COMPACT_STRING
    group_instance_id => COMPACT_NULLABLE_STRING
    metadata => COMPACT_BYTES
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。
generation_idグループの生成ID。
protocol_nameコーディネータによって選択されたグループ プロトコル。
leaderグループのリーダー。
member_idグループ コーディネータによって割り当てられたメンバーID。
members
member_idグループ メンバー ID。
group_instance_idエンドユーザによって提供されるコンシューマ インスタンスの一意の識別子。
metadataグループ メンバーのメタデータ。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

JoinGroup Response (Version: 7) => throttle_time_ms error_code generation_id protocol_type protocol_name leader member_id [members] TAG_BUFFER 
  throttle_time_ms => INT32
  error_code => INT16
  generation_id => INT32
  protocol_type => COMPACT_NULLABLE_STRING
  protocol_name => COMPACT_NULLABLE_STRING
  leader => COMPACT_STRING
  member_id => COMPACT_STRING
  members => member_id group_instance_id metadata TAG_BUFFER 
    member_id => COMPACT_STRING
    group_instance_id => COMPACT_NULLABLE_STRING
    metadata => COMPACT_BYTES
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。
generation_idグループの生成ID。
protocol_typeグループプロトコル名。
protocol_nameコーディネータによって選択されたグループ プロトコル。
leaderグループのリーダー。
member_idグループ コーディネータによって割り当てられたメンバーID。
members
member_idグループ メンバー ID。
group_instance_idエンドユーザによって提供されるコンシューマ インスタンスの一意の識別子。
metadataグループ メンバーのメタデータ。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

Heartbeat API (Key: 12):
Requests:

Heartbeat Request (Version: 0) => group_id generation_id member_id 
  group_id => STRING
  generation_id => INT32
  member_id => STRING
フィールド 説明
group_idグループ id。
generation_idグループの世代。
member_idメンバーID。

Heartbeat Request (Version: 1) => group_id generation_id member_id 
  group_id => STRING
  generation_id => INT32
  member_id => STRING
フィールド 説明
group_idグループ id。
generation_idグループの世代。
member_idメンバーID。

Heartbeat Request (Version: 2) => group_id generation_id member_id 
  group_id => STRING
  generation_id => INT32
  member_id => STRING
フィールド 説明
group_idグループ id。
generation_idグループの世代。
member_idメンバーID。

Heartbeat Request (Version: 3) => group_id generation_id member_id group_instance_id 
  group_id => STRING
  generation_id => INT32
  member_id => STRING
  group_instance_id => NULLABLE_STRING
フィールド 説明
group_idグループ id。
generation_idグループの世代。
member_idメンバーID。
group_instance_idエンドユーザによって提供されるコンシューマ インスタンスの一意の識別子。

Heartbeat Request (Version: 4) => group_id generation_id member_id group_instance_id TAG_BUFFER 
  group_id => COMPACT_STRING
  generation_id => INT32
  member_id => COMPACT_STRING
  group_instance_id => COMPACT_NULLABLE_STRING
フィールド 説明
group_idグループ id。
generation_idグループの世代。
member_idメンバーID。
group_instance_idエンドユーザによって提供されるコンシューマ インスタンスの一意の識別子。
_tagged_fieldsタグ付きのフィールド

Responses:

Heartbeat Response (Version: 0) => error_code 
  error_code => INT16
フィールド 説明
error_codeエラーコード。あるいはエラーが無かった場合は0。

Heartbeat Response (Version: 1) => throttle_time_ms error_code 
  throttle_time_ms => INT32
  error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。

Heartbeat Response (Version: 2) => throttle_time_ms error_code 
  throttle_time_ms => INT32
  error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。

Heartbeat Response (Version: 3) => throttle_time_ms error_code 
  throttle_time_ms => INT32
  error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。

Heartbeat Response (Version: 4) => throttle_time_ms error_code TAG_BUFFER 
  throttle_time_ms => INT32
  error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。
_tagged_fieldsタグ付きのフィールド

LeaveGroup API (Key: 13):
Requests:

LeaveGroup Request (Version: 0) => group_id member_id 
  group_id => STRING
  member_id => STRING
フィールド 説明
group_id離れるグループのID。
member_idグループから削除するメンバーID。

LeaveGroup Request (Version: 1) => group_id member_id 
  group_id => STRING
  member_id => STRING
フィールド 説明
group_id離れるグループのID。
member_idグループから削除するメンバーID。

LeaveGroup Request (Version: 2) => group_id member_id 
  group_id => STRING
  member_id => STRING
フィールド 説明
group_id離れるグループのID。
member_idグループから削除するメンバーID。

LeaveGroup Request (Version: 3) => group_id [members] 
  group_id => STRING
  members => member_id group_instance_id 
    member_id => STRING
    group_instance_id => NULLABLE_STRING
フィールド 説明
group_id離れるグループのID。
members退会メンバー ID のリスト。
member_idグループから削除するメンバーID。
group_instance_idグループから削除するグループインスタンス ID。

LeaveGroup Request (Version: 4) => group_id [members] TAG_BUFFER 
  group_id => COMPACT_STRING
  members => member_id group_instance_id TAG_BUFFER 
    member_id => COMPACT_STRING
    group_instance_id => COMPACT_NULLABLE_STRING
フィールド 説明
group_id離れるグループのID。
members退会メンバー ID のリスト。
member_idグループから削除するメンバーID。
group_instance_idグループから削除するグループインスタンス ID。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

Responses:

LeaveGroup Response (Version: 0) => error_code 
  error_code => INT16
フィールド 説明
error_codeエラーコード。あるいはエラーが無かった場合は0。

LeaveGroup Response (Version: 1) => throttle_time_ms error_code 
  throttle_time_ms => INT32
  error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。

LeaveGroup Response (Version: 2) => throttle_time_ms error_code 
  throttle_time_ms => INT32
  error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。

LeaveGroup Response (Version: 3) => throttle_time_ms error_code [members] 
  throttle_time_ms => INT32
  error_code => INT16
  members => member_id group_instance_id error_code 
    member_id => STRING
    group_instance_id => NULLABLE_STRING
    error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。
members退会メンバー応答のリスト。
member_idグループから削除するメンバーID。
group_instance_idグループから削除するグループインスタンス ID。
error_codeエラーコード。あるいはエラーが無かった場合は0。

LeaveGroup Response (Version: 4) => throttle_time_ms error_code [members] TAG_BUFFER 
  throttle_time_ms => INT32
  error_code => INT16
  members => member_id group_instance_id error_code TAG_BUFFER 
    member_id => COMPACT_STRING
    group_instance_id => COMPACT_NULLABLE_STRING
    error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。
members退会メンバー応答のリスト。
member_idグループから削除するメンバーID。
group_instance_idグループから削除するグループインスタンス ID。
error_codeエラーコード。あるいはエラーが無かった場合は0。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

SyncGroup API (Key: 14):
Requests:

SyncGroup Request (Version: 0) => group_id generation_id member_id [assignments] 
  group_id => STRING
  generation_id => INT32
  member_id => STRING
  assignments => member_id assignment 
    member_id => STRING
    assignment => BYTES
フィールド 説明
group_idユニークなグループの識別子。
generation_idグループの世代。
member_idグループによって割り当てられたメンバーID。
assignments各割り当て。
member_id割り当てるメンバーのID。
assignmentメンバーの割り当て。

SyncGroup Request (Version: 1) => group_id generation_id member_id [assignments] 
  group_id => STRING
  generation_id => INT32
  member_id => STRING
  assignments => member_id assignment 
    member_id => STRING
    assignment => BYTES
フィールド 説明
group_idユニークなグループの識別子。
generation_idグループの世代。
member_idグループによって割り当てられたメンバーID。
assignments各割り当て。
member_id割り当てるメンバーのID。
assignmentメンバーの割り当て。

SyncGroup Request (Version: 2) => group_id generation_id member_id [assignments] 
  group_id => STRING
  generation_id => INT32
  member_id => STRING
  assignments => member_id assignment 
    member_id => STRING
    assignment => BYTES
フィールド 説明
group_idユニークなグループの識別子。
generation_idグループの世代。
member_idグループによって割り当てられたメンバーID。
assignments各割り当て。
member_id割り当てるメンバーのID。
assignmentメンバーの割り当て。

SyncGroup Request (Version: 3) => group_id generation_id member_id group_instance_id [assignments] 
  group_id => STRING
  generation_id => INT32
  member_id => STRING
  group_instance_id => NULLABLE_STRING
  assignments => member_id assignment 
    member_id => STRING
    assignment => BYTES
フィールド 説明
group_idユニークなグループの識別子。
generation_idグループの世代。
member_idグループによって割り当てられたメンバーID。
group_instance_idエンドユーザによって提供されるコンシューマ インスタンスの一意の識別子。
assignments各割り当て。
member_id割り当てるメンバーのID。
assignmentメンバーの割り当て。

SyncGroup Request (Version: 4) => group_id generation_id member_id group_instance_id [assignments] TAG_BUFFER 
  group_id => COMPACT_STRING
  generation_id => INT32
  member_id => COMPACT_STRING
  group_instance_id => COMPACT_NULLABLE_STRING
  assignments => member_id assignment TAG_BUFFER 
    member_id => COMPACT_STRING
    assignment => COMPACT_BYTES
フィールド 説明
group_idユニークなグループの識別子。
generation_idグループの世代。
member_idグループによって割り当てられたメンバーID。
group_instance_idエンドユーザによって提供されるコンシューマ インスタンスの一意の識別子。
assignments各割り当て。
member_id割り当てるメンバーのID。
assignmentメンバーの割り当て。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

SyncGroup Request (Version: 5) => group_id generation_id member_id group_instance_id protocol_type protocol_name [assignments] TAG_BUFFER 
  group_id => COMPACT_STRING
  generation_id => INT32
  member_id => COMPACT_STRING
  group_instance_id => COMPACT_NULLABLE_STRING
  protocol_type => COMPACT_NULLABLE_STRING
  protocol_name => COMPACT_NULLABLE_STRING
  assignments => member_id assignment TAG_BUFFER 
    member_id => COMPACT_STRING
    assignment => COMPACT_BYTES
フィールド 説明
group_idユニークなグループの識別子。
generation_idグループの世代。
member_idグループによって割り当てられたメンバーID。
group_instance_idエンドユーザによって提供されるコンシューマ インスタンスの一意の識別子。
protocol_typeグループプロトコルの型。
protocol_nameグループプロトコル名。
assignments各割り当て。
member_id割り当てるメンバーのID。
assignmentメンバーの割り当て。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

Responses:

SyncGroup Response (Version: 0) => error_code assignment 
  error_code => INT16
  assignment => BYTES
フィールド 説明
error_codeエラーコード。あるいはエラーが無かった場合は0。
assignmentメンバーの割り当て。

SyncGroup Response (Version: 1) => throttle_time_ms error_code assignment 
  throttle_time_ms => INT32
  error_code => INT16
  assignment => BYTES
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。
assignmentメンバーの割り当て。

SyncGroup Response (Version: 2) => throttle_time_ms error_code assignment 
  throttle_time_ms => INT32
  error_code => INT16
  assignment => BYTES
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。
assignmentメンバーの割り当て。

SyncGroup Response (Version: 3) => throttle_time_ms error_code assignment 
  throttle_time_ms => INT32
  error_code => INT16
  assignment => BYTES
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。
assignmentメンバーの割り当て。

SyncGroup Response (Version: 4) => throttle_time_ms error_code assignment TAG_BUFFER 
  throttle_time_ms => INT32
  error_code => INT16
  assignment => COMPACT_BYTES
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。
assignmentメンバーの割り当て。
_tagged_fieldsタグ付きのフィールド

SyncGroup Response (Version: 5) => throttle_time_ms error_code protocol_type protocol_name assignment TAG_BUFFER 
  throttle_time_ms => INT32
  error_code => INT16
  protocol_type => COMPACT_NULLABLE_STRING
  protocol_name => COMPACT_NULLABLE_STRING
  assignment => COMPACT_BYTES
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。
protocol_typeグループプロトコルの型。
protocol_nameグループプロトコル名。
assignmentメンバーの割り当て。
_tagged_fieldsタグ付きのフィールド

DescribeGroups API (Key: 15):
Requests:

DescribeGroups Request (Version: 0) => [groups] 
  groups => STRING
フィールド 説明
groups記述するグループの名前

DescribeGroups Request (Version: 1) => [groups] 
  groups => STRING
フィールド 説明
groups記述するグループの名前

DescribeGroups Request (Version: 2) => [groups] 
  groups => STRING
フィールド 説明
groups記述するグループの名前

DescribeGroups Request (Version: 3) => [groups] include_authorized_operations 
  groups => STRING
  include_authorized_operations => BOOLEAN
フィールド 説明
groups記述するグループの名前
include_authorized_operations認証操作を含むかどうか。

DescribeGroups Request (Version: 4) => [groups] include_authorized_operations 
  groups => STRING
  include_authorized_operations => BOOLEAN
フィールド 説明
groups記述するグループの名前
include_authorized_operations認証操作を含むかどうか。

DescribeGroups Request (Version: 5) => [groups] include_authorized_operations TAG_BUFFER 
  groups => COMPACT_STRING
  include_authorized_operations => BOOLEAN
フィールド 説明
groups記述するグループの名前
include_authorized_operations認証操作を含むかどうか。
_tagged_fieldsタグ付きのフィールド

Responses:

DescribeGroups Response (Version: 0) => [groups] 
  groups => error_code group_id group_state protocol_type protocol_data [members] 
    error_code => INT16
    group_id => STRING
    group_state => STRING
    protocol_type => STRING
    protocol_data => STRING
    members => member_id client_id client_host member_metadata member_assignment 
      member_id => STRING
      client_id => STRING
      client_host => STRING
      member_metadata => BYTES
      member_assignment => BYTES
フィールド 説明
groups記述される各グループ。
error_code記述エラー。あるいはエラーが無かった場合は0。
group_idグループID文字列。
group_stateグループの状態文字列。あるいは空の文字列。
protocol_typeグループ プロトコルの型。あるいは空の文字列。
protocol_dataグループプロトコル データ。あるいは空の文字列。
membersグループのメンバー。
member_idグループ コーディネータによって割り当てられたメンバーID。
client_idメンバーの最新の参加グループ リクエストの中で使われる クライアントID。
client_hostクライアント ホスト。
member_metadata使用中の現在のグループプロトコルに対応するメタデータ。
member_assignmentグループリーダーによって提供される現在の割り当て。

DescribeGroups Response (Version: 1) => throttle_time_ms [groups] 
  throttle_time_ms => INT32
  groups => error_code group_id group_state protocol_type protocol_data [members] 
    error_code => INT16
    group_id => STRING
    group_state => STRING
    protocol_type => STRING
    protocol_data => STRING
    members => member_id client_id client_host member_metadata member_assignment 
      member_id => STRING
      client_id => STRING
      client_host => STRING
      member_metadata => BYTES
      member_assignment => BYTES
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
groups記述される各グループ。
error_code記述エラー。あるいはエラーが無かった場合は0。
group_idグループID文字列。
group_stateグループの状態文字列。あるいは空の文字列。
protocol_typeグループ プロトコルの型。あるいは空の文字列。
protocol_dataグループプロトコル データ。あるいは空の文字列。
membersグループのメンバー。
member_idグループ コーディネータによって割り当てられたメンバーID。
client_idメンバーの最新の参加グループ リクエストの中で使われる クライアントID。
client_hostクライアント ホスト。
member_metadata使用中の現在のグループプロトコルに対応するメタデータ。
member_assignmentグループリーダーによって提供される現在の割り当て。

DescribeGroups Response (Version: 2) => throttle_time_ms [groups] 
  throttle_time_ms => INT32
  groups => error_code group_id group_state protocol_type protocol_data [members] 
    error_code => INT16
    group_id => STRING
    group_state => STRING
    protocol_type => STRING
    protocol_data => STRING
    members => member_id client_id client_host member_metadata member_assignment 
      member_id => STRING
      client_id => STRING
      client_host => STRING
      member_metadata => BYTES
      member_assignment => BYTES
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
groups記述される各グループ。
error_code記述エラー。あるいはエラーが無かった場合は0。
group_idグループID文字列。
group_stateグループの状態文字列。あるいは空の文字列。
protocol_typeグループ プロトコルの型。あるいは空の文字列。
protocol_dataグループプロトコル データ。あるいは空の文字列。
membersグループのメンバー。
member_idグループ コーディネータによって割り当てられたメンバーID。
client_idメンバーの最新の参加グループ リクエストの中で使われる クライアントID。
client_hostクライアント ホスト。
member_metadata使用中の現在のグループプロトコルに対応するメタデータ。
member_assignmentグループリーダーによって提供される現在の割り当て。

DescribeGroups Response (Version: 3) => throttle_time_ms [groups] 
  throttle_time_ms => INT32
  groups => error_code group_id group_state protocol_type protocol_data [members] authorized_operations 
    error_code => INT16
    group_id => STRING
    group_state => STRING
    protocol_type => STRING
    protocol_data => STRING
    members => member_id client_id client_host member_metadata member_assignment 
      member_id => STRING
      client_id => STRING
      client_host => STRING
      member_metadata => BYTES
      member_assignment => BYTES
    authorized_operations => INT32
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
groups記述される各グループ。
error_code記述エラー。あるいはエラーが無かった場合は0。
group_idグループID文字列。
group_stateグループの状態文字列。あるいは空の文字列。
protocol_typeグループ プロトコルの型。あるいは空の文字列。
protocol_dataグループプロトコル データ。あるいは空の文字列。
membersグループのメンバー。
member_idグループ コーディネータによって割り当てられたメンバーID。
client_idメンバーの最新の参加グループ リクエストの中で使われる クライアントID。
client_hostクライアント ホスト。
member_metadata使用中の現在のグループプロトコルに対応するメタデータ。
member_assignmentグループリーダーによって提供される現在の割り当て。
authorized_operationsこのグループについて認証された操作を表す32ビットのビットフィールド。

DescribeGroups Response (Version: 4) => throttle_time_ms [groups] 
  throttle_time_ms => INT32
  groups => error_code group_id group_state protocol_type protocol_data [members] authorized_operations 
    error_code => INT16
    group_id => STRING
    group_state => STRING
    protocol_type => STRING
    protocol_data => STRING
    members => member_id group_instance_id client_id client_host member_metadata member_assignment 
      member_id => STRING
      group_instance_id => NULLABLE_STRING
      client_id => STRING
      client_host => STRING
      member_metadata => BYTES
      member_assignment => BYTES
    authorized_operations => INT32
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
groups記述される各グループ。
error_code記述エラー。あるいはエラーが無かった場合は0。
group_idグループID文字列。
group_stateグループの状態文字列。あるいは空の文字列。
protocol_typeグループ プロトコルの型。あるいは空の文字列。
protocol_dataグループプロトコル データ。あるいは空の文字列。
membersグループのメンバー。
member_idグループ コーディネータによって割り当てられたメンバーID。
group_instance_idエンドユーザによって提供されるコンシューマ インスタンスの一意の識別子。
client_idメンバーの最新の参加グループ リクエストの中で使われる クライアントID。
client_hostクライアント ホスト。
member_metadata使用中の現在のグループプロトコルに対応するメタデータ。
member_assignmentグループリーダーによって提供される現在の割り当て。
authorized_operationsこのグループについて認証された操作を表す32ビットのビットフィールド。

DescribeGroups Response (Version: 5) => throttle_time_ms [groups] TAG_BUFFER 
  throttle_time_ms => INT32
  groups => error_code group_id group_state protocol_type protocol_data [members] authorized_operations TAG_BUFFER 
    error_code => INT16
    group_id => COMPACT_STRING
    group_state => COMPACT_STRING
    protocol_type => COMPACT_STRING
    protocol_data => COMPACT_STRING
    members => member_id group_instance_id client_id client_host member_metadata member_assignment TAG_BUFFER 
      member_id => COMPACT_STRING
      group_instance_id => COMPACT_NULLABLE_STRING
      client_id => COMPACT_STRING
      client_host => COMPACT_STRING
      member_metadata => COMPACT_BYTES
      member_assignment => COMPACT_BYTES
    authorized_operations => INT32
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
groups記述される各グループ。
error_code記述エラー。あるいはエラーが無かった場合は0。
group_idグループID文字列。
group_stateグループの状態文字列。あるいは空の文字列。
protocol_typeグループ プロトコルの型。あるいは空の文字列。
protocol_dataグループプロトコル データ。あるいは空の文字列。
membersグループのメンバー。
member_idグループ コーディネータによって割り当てられたメンバーID。
group_instance_idエンドユーザによって提供されるコンシューマ インスタンスの一意の識別子。
client_idメンバーの最新の参加グループ リクエストの中で使われる クライアントID。
client_hostクライアント ホスト。
member_metadata使用中の現在のグループプロトコルに対応するメタデータ。
member_assignmentグループリーダーによって提供される現在の割り当て。
_tagged_fieldsタグ付きのフィールド
authorized_operationsこのグループについて認証された操作を表す32ビットのビットフィールド。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

ListGroups API (Key: 16):
Requests:

ListGroups Request (Version: 0) => 
フィールド 説明

ListGroups Request (Version: 1) => 
フィールド 説明

ListGroups Request (Version: 2) => 
フィールド 説明

ListGroups Request (Version: 3) => TAG_BUFFER 
フィールド 説明
_tagged_fieldsタグ付きのフィールド

ListGroups Request (Version: 4) => [states_filter] TAG_BUFFER 
  states_filter => COMPACT_STRING
フィールド 説明
states_filterリストするグループの状態。空の場合、全てのグループがその状態を返します。
_tagged_fieldsタグ付きのフィールド

Responses:

ListGroups Response (Version: 0) => error_code [groups] 
  error_code => INT16
  groups => group_id protocol_type 
    group_id => STRING
    protocol_type => STRING
フィールド 説明
error_codeエラーコード。あるいはエラーが無かった場合は0。
groups応答の中の各グループ。
group_idグループ ID。
protocol_typeグループプロトコルの型。

ListGroups Response (Version: 1) => throttle_time_ms error_code [groups] 
  throttle_time_ms => INT32
  error_code => INT16
  groups => group_id protocol_type 
    group_id => STRING
    protocol_type => STRING
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。
groups応答の中の各グループ。
group_idグループ ID。
protocol_typeグループプロトコルの型。

ListGroups Response (Version: 2) => throttle_time_ms error_code [groups] 
  throttle_time_ms => INT32
  error_code => INT16
  groups => group_id protocol_type 
    group_id => STRING
    protocol_type => STRING
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。
groups応答の中の各グループ。
group_idグループ ID。
protocol_typeグループプロトコルの型。

ListGroups Response (Version: 3) => throttle_time_ms error_code [groups] TAG_BUFFER 
  throttle_time_ms => INT32
  error_code => INT16
  groups => group_id protocol_type TAG_BUFFER 
    group_id => COMPACT_STRING
    protocol_type => COMPACT_STRING
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。
groups応答の中の各グループ。
group_idグループ ID。
protocol_typeグループプロトコルの型。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

ListGroups Response (Version: 4) => throttle_time_ms error_code [groups] TAG_BUFFER 
  throttle_time_ms => INT32
  error_code => INT16
  groups => group_id protocol_type group_state TAG_BUFFER 
    group_id => COMPACT_STRING
    protocol_type => COMPACT_STRING
    group_state => COMPACT_STRING
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。
groups応答の中の各グループ。
group_idグループ ID。
protocol_typeグループプロトコルの型。
group_stateグループ状態名。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

SaslHandshake API (Key: 17):
Requests:

SaslHandshake Request (Version: 0) => mechanism 
  mechanism => STRING
フィールド 説明
mechanismクライアントによって選択されたSASL機構。

SaslHandshake Request (Version: 1) => mechanism 
  mechanism => STRING
フィールド 説明
mechanismクライアントによって選択されたSASL機構。

Responses:

SaslHandshake Response (Version: 0) => error_code [mechanisms] 
  error_code => INT16
  mechanisms => STRING
フィールド 説明
error_codeエラーコード。あるいはエラーが無かった場合は0。
mechanismsサーバ内で有効にされた機構。

SaslHandshake Response (Version: 1) => error_code [mechanisms] 
  error_code => INT16
  mechanisms => STRING
フィールド 説明
error_codeエラーコード。あるいはエラーが無かった場合は0。
mechanismsサーバ内で有効にされた機構。

ApiVersions API (Key: 18):
Requests:

ApiVersions Request (Version: 0) => 
フィールド 説明

ApiVersions Request (Version: 1) => 
フィールド 説明

ApiVersions Request (Version: 2) => 
フィールド 説明

ApiVersions Request (Version: 3) => client_software_name client_software_version TAG_BUFFER 
  client_software_name => COMPACT_STRING
  client_software_version => COMPACT_STRING
フィールド 説明
client_software_nameクライアント名。
client_software_versionクライアントのバージョン。
_tagged_fieldsタグ付きのフィールド

Responses:

ApiVersions Response (Version: 0) => error_code [api_keys] 
  error_code => INT16
  api_keys => api_key min_version max_version 
    api_key => INT16
    min_version => INT16
    max_version => INT16
フィールド 説明
error_codeトップレベルのエラーコード。
api_keysブローカーによってサポートされる API。
api_keyAPI のインデックス。
min_versionサポートされる最小バージョン(含まれる)。
max_versionサポートされる最大バージョン(含まれる)。

ApiVersions Response (Version: 1) => error_code [api_keys] throttle_time_ms 
  error_code => INT16
  api_keys => api_key min_version max_version 
    api_key => INT16
    min_version => INT16
    max_version => INT16
  throttle_time_ms => INT32
フィールド 説明
error_codeトップレベルのエラーコード。
api_keysブローカーによってサポートされる API。
api_keyAPI のインデックス。
min_versionサポートされる最小バージョン(含まれる)。
max_versionサポートされる最大バージョン(含まれる)。
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。

ApiVersions Response (Version: 2) => error_code [api_keys] throttle_time_ms 
  error_code => INT16
  api_keys => api_key min_version max_version 
    api_key => INT16
    min_version => INT16
    max_version => INT16
  throttle_time_ms => INT32
フィールド 説明
error_codeトップレベルのエラーコード。
api_keysブローカーによってサポートされる API。
api_keyAPI のインデックス。
min_versionサポートされる最小バージョン(含まれる)。
max_versionサポートされる最大バージョン(含まれる)。
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。

ApiVersions Response (Version: 3) => error_code [api_keys] throttle_time_ms TAG_BUFFER 
  error_code => INT16
  api_keys => api_key min_version max_version TAG_BUFFER 
    api_key => INT16
    min_version => INT16
    max_version => INT16
  throttle_time_ms => INT32
フィールド 説明
error_codeトップレベルのエラーコード。
api_keysブローカーによってサポートされる API。
api_keyAPI のインデックス。
min_versionサポートされる最小バージョン(含まれる)。
max_versionサポートされる最大バージョン(含まれる)。
_tagged_fieldsタグ付きのフィールド
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
_tagged_fieldsタグ付きのフィールド

CreateTopics API (Key: 19):
Requests:

CreateTopics Request (Version: 0) => [topics] timeout_ms 
  topics => name num_partitions replication_factor [assignments] [configs] 
    name => STRING
    num_partitions => INT32
    replication_factor => INT16
    assignments => partition_index [broker_ids] 
      partition_index => INT32
      broker_ids => INT32
    configs => name value 
      name => STRING
      value => NULLABLE_STRING
  timeout_ms => INT32
フィールド 説明
topics作成するトピック。
名前トピック名。
num_partitionsトピックで作成するパーティション数。または、手動パーティション割り当てを指定するか、デフォルトのパーティションを使う場合は -1。
replication_factorトピック内の各パーティションに作成するレプリカの数。または、手動パーティション割り当てを指定するか、デフォルトのレプリケーション係数を使う場合は -1。
assignments手動のパーティション割り当て。あるいは自動的な割り当てを使っている場合は空の配列。
partition_indexパーティションのインデックス。
broker_idsパーティションを配置するブローカー。
configs設定のための独自のトピック設定。
名前設定名。
設定値。
timeout_msリクエストをタイムアウトする前にどれだけ長く待つかのミリ秒。

CreateTopics Request (Version: 1) => [topics] timeout_ms validate_only 
  topics => name num_partitions replication_factor [assignments] [configs] 
    name => STRING
    num_partitions => INT32
    replication_factor => INT16
    assignments => partition_index [broker_ids] 
      partition_index => INT32
      broker_ids => INT32
    configs => name value 
      name => STRING
      value => NULLABLE_STRING
  timeout_ms => INT32
  validate_only => BOOLEAN
フィールド 説明
topics作成するトピック。
名前トピック名。
num_partitionsトピックで作成するパーティション数。または、手動パーティション割り当てを指定するか、デフォルトのパーティションを使う場合は -1。
replication_factorトピック内の各パーティションに作成するレプリカの数。または、手動パーティション割り当てを指定するか、デフォルトのレプリケーション係数を使う場合は -1。
assignments手動のパーティション割り当て。あるいは自動的な割り当てを使っている場合は空の配列。
partition_indexパーティションのインデックス。
broker_idsパーティションを配置するブローカー。
configs設定のための独自のトピック設定。
名前設定名。
設定値。
timeout_msリクエストをタイムアウトする前にどれだけ長く待つかのミリ秒。
validate_onlytrueであれば、指定されたようにトピックが作成できるかを調べますが、何も作成しません。

CreateTopics Request (Version: 2) => [topics] timeout_ms validate_only 
  topics => name num_partitions replication_factor [assignments] [configs] 
    name => STRING
    num_partitions => INT32
    replication_factor => INT16
    assignments => partition_index [broker_ids] 
      partition_index => INT32
      broker_ids => INT32
    configs => name value 
      name => STRING
      value => NULLABLE_STRING
  timeout_ms => INT32
  validate_only => BOOLEAN
フィールド 説明
topics作成するトピック。
名前トピック名。
num_partitionsトピックで作成するパーティション数。または、手動パーティション割り当てを指定するか、デフォルトのパーティションを使う場合は -1。
replication_factorトピック内の各パーティションに作成するレプリカの数。または、手動パーティション割り当てを指定するか、デフォルトのレプリケーション係数を使う場合は -1。
assignments手動のパーティション割り当て。あるいは自動的な割り当てを使っている場合は空の配列。
partition_indexパーティションのインデックス。
broker_idsパーティションを配置するブローカー。
configs設定のための独自のトピック設定。
名前設定名。
設定値。
timeout_msリクエストをタイムアウトする前にどれだけ長く待つかのミリ秒。
validate_onlytrueであれば、指定されたようにトピックが作成できるかを調べますが、何も作成しません。

CreateTopics Request (Version: 3) => [topics] timeout_ms validate_only 
  topics => name num_partitions replication_factor [assignments] [configs] 
    name => STRING
    num_partitions => INT32
    replication_factor => INT16
    assignments => partition_index [broker_ids] 
      partition_index => INT32
      broker_ids => INT32
    configs => name value 
      name => STRING
      value => NULLABLE_STRING
  timeout_ms => INT32
  validate_only => BOOLEAN
フィールド 説明
topics作成するトピック。
名前トピック名。
num_partitionsトピックで作成するパーティション数。または、手動パーティション割り当てを指定するか、デフォルトのパーティションを使う場合は -1。
replication_factorトピック内の各パーティションに作成するレプリカの数。または、手動パーティション割り当てを指定するか、デフォルトのレプリケーション係数を使う場合は -1。
assignments手動のパーティション割り当て。あるいは自動的な割り当てを使っている場合は空の配列。
partition_indexパーティションのインデックス。
broker_idsパーティションを配置するブローカー。
configs設定のための独自のトピック設定。
名前設定名。
設定値。
timeout_msリクエストをタイムアウトする前にどれだけ長く待つかのミリ秒。
validate_onlytrueであれば、指定されたようにトピックが作成できるかを調べますが、何も作成しません。

CreateTopics Request (Version: 4) => [topics] timeout_ms validate_only 
  topics => name num_partitions replication_factor [assignments] [configs] 
    name => STRING
    num_partitions => INT32
    replication_factor => INT16
    assignments => partition_index [broker_ids] 
      partition_index => INT32
      broker_ids => INT32
    configs => name value 
      name => STRING
      value => NULLABLE_STRING
  timeout_ms => INT32
  validate_only => BOOLEAN
フィールド 説明
topics作成するトピック。
名前トピック名。
num_partitionsトピックで作成するパーティション数。または、手動パーティション割り当てを指定するか、デフォルトのパーティションを使う場合は -1。
replication_factorトピック内の各パーティションに作成するレプリカの数。または、手動パーティション割り当てを指定するか、デフォルトのレプリケーション係数を使う場合は -1。
assignments手動のパーティション割り当て。あるいは自動的な割り当てを使っている場合は空の配列。
partition_indexパーティションのインデックス。
broker_idsパーティションを配置するブローカー。
configs設定のための独自のトピック設定。
名前設定名。
設定値。
timeout_msリクエストをタイムアウトする前にどれだけ長く待つかのミリ秒。
validate_onlytrueであれば、指定されたようにトピックが作成できるかを調べますが、何も作成しません。

CreateTopics Request (Version: 5) => [topics] timeout_ms validate_only TAG_BUFFER 
  topics => name num_partitions replication_factor [assignments] [configs] TAG_BUFFER 
    name => COMPACT_STRING
    num_partitions => INT32
    replication_factor => INT16
    assignments => partition_index [broker_ids] TAG_BUFFER 
      partition_index => INT32
      broker_ids => INT32
    configs => name value TAG_BUFFER 
      name => COMPACT_STRING
      value => COMPACT_NULLABLE_STRING
  timeout_ms => INT32
  validate_only => BOOLEAN
フィールド 説明
topics作成するトピック。
名前トピック名。
num_partitionsトピックで作成するパーティション数。または、手動パーティション割り当てを指定するか、デフォルトのパーティションを使う場合は -1。
replication_factorトピック内の各パーティションに作成するレプリカの数。または、手動パーティション割り当てを指定するか、デフォルトのレプリケーション係数を使う場合は -1。
assignments手動のパーティション割り当て。あるいは自動的な割り当てを使っている場合は空の配列。
partition_indexパーティションのインデックス。
broker_idsパーティションを配置するブローカー。
_tagged_fieldsタグ付きのフィールド
configs設定のための独自のトピック設定。
名前設定名。
設定値。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド
timeout_msリクエストをタイムアウトする前にどれだけ長く待つかのミリ秒。
validate_onlytrueであれば、指定されたようにトピックが作成できるかを調べますが、何も作成しません。
_tagged_fieldsタグ付きのフィールド

CreateTopics Request (Version: 6) => [topics] timeout_ms validate_only TAG_BUFFER 
  topics => name num_partitions replication_factor [assignments] [configs] TAG_BUFFER 
    name => COMPACT_STRING
    num_partitions => INT32
    replication_factor => INT16
    assignments => partition_index [broker_ids] TAG_BUFFER 
      partition_index => INT32
      broker_ids => INT32
    configs => name value TAG_BUFFER 
      name => COMPACT_STRING
      value => COMPACT_NULLABLE_STRING
  timeout_ms => INT32
  validate_only => BOOLEAN
フィールド 説明
topics作成するトピック。
名前トピック名。
num_partitionsトピックで作成するパーティション数。または、手動パーティション割り当てを指定するか、デフォルトのパーティションを使う場合は -1。
replication_factorトピック内の各パーティションに作成するレプリカの数。または、手動パーティション割り当てを指定するか、デフォルトのレプリケーション係数を使う場合は -1。
assignments手動のパーティション割り当て。あるいは自動的な割り当てを使っている場合は空の配列。
partition_indexパーティションのインデックス。
broker_idsパーティションを配置するブローカー。
_tagged_fieldsタグ付きのフィールド
configs設定のための独自のトピック設定。
名前設定名。
設定値。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド
timeout_msリクエストをタイムアウトする前にどれだけ長く待つかのミリ秒。
validate_onlytrueであれば、指定されたようにトピックが作成できるかを調べますが、何も作成しません。
_tagged_fieldsタグ付きのフィールド

Responses:

CreateTopics Response (Version: 0) => [topics] 
  topics => name error_code 
    name => STRING
    error_code => INT16
フィールド 説明
topics作成しようとした各トピックの結果。
名前トピック名。
error_codeエラーコード。あるいはエラーが無かった場合は0。

CreateTopics Response (Version: 1) => [topics] 
  topics => name error_code error_message 
    name => STRING
    error_code => INT16
    error_message => NULLABLE_STRING
フィールド 説明
topics作成しようとした各トピックの結果。
名前トピック名。
error_codeエラーコード。あるいはエラーが無かった場合は0。
error_messageエラーメッセージ。あるいはエラーが無かった場合はnull。

CreateTopics Response (Version: 2) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => name error_code error_message 
    name => STRING
    error_code => INT16
    error_message => NULLABLE_STRING
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
topics作成しようとした各トピックの結果。
名前トピック名。
error_codeエラーコード。あるいはエラーが無かった場合は0。
error_messageエラーメッセージ。あるいはエラーが無かった場合はnull。

CreateTopics Response (Version: 3) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => name error_code error_message 
    name => STRING
    error_code => INT16
    error_message => NULLABLE_STRING
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
topics作成しようとした各トピックの結果。
名前トピック名。
error_codeエラーコード。あるいはエラーが無かった場合は0。
error_messageエラーメッセージ。あるいはエラーが無かった場合はnull。

CreateTopics Response (Version: 4) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => name error_code error_message 
    name => STRING
    error_code => INT16
    error_message => NULLABLE_STRING
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
topics作成しようとした各トピックの結果。
名前トピック名。
error_codeエラーコード。あるいはエラーが無かった場合は0。
error_messageエラーメッセージ。あるいはエラーが無かった場合はnull。

CreateTopics Response (Version: 5) => throttle_time_ms [topics] TAG_BUFFER 
  throttle_time_ms => INT32
  topics => name error_code error_message num_partitions replication_factor [configs] TAG_BUFFER 
    name => COMPACT_STRING
    error_code => INT16
    error_message => COMPACT_NULLABLE_STRING
    num_partitions => INT32
    replication_factor => INT16
    configs => name value read_only config_source is_sensitive TAG_BUFFER 
      name => COMPACT_STRING
      value => COMPACT_NULLABLE_STRING
      read_only => BOOLEAN
      config_source => INT8
      is_sensitive => BOOLEAN
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
topics作成しようとした各トピックの結果。
名前トピック名。
error_codeエラーコード。あるいはエラーが無かった場合は0。
error_messageエラーメッセージ。あるいはエラーが無かった場合はnull。
num_partitionsトピックのパーティション数。
replication_factorトピックのレプリケーション係数。
configsトピックの設定。
名前設定名。
設定値。
read_only設定が読み込み専用の場合は True。
config_source設定ソース。
is_sensitiveTrue if this configuration is sensitive.
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

CreateTopics Response (Version: 6) => throttle_time_ms [topics] TAG_BUFFER 
  throttle_time_ms => INT32
  topics => name error_code error_message num_partitions replication_factor [configs] TAG_BUFFER 
    name => COMPACT_STRING
    error_code => INT16
    error_message => COMPACT_NULLABLE_STRING
    num_partitions => INT32
    replication_factor => INT16
    configs => name value read_only config_source is_sensitive TAG_BUFFER 
      name => COMPACT_STRING
      value => COMPACT_NULLABLE_STRING
      read_only => BOOLEAN
      config_source => INT8
      is_sensitive => BOOLEAN
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
topics作成しようとした各トピックの結果。
名前トピック名。
error_codeエラーコード。あるいはエラーが無かった場合は0。
error_messageエラーメッセージ。あるいはエラーが無かった場合はnull。
num_partitionsトピックのパーティション数。
replication_factorトピックのレプリケーション係数。
configsトピックの設定。
名前設定名。
設定値。
read_only設定が読み込み専用の場合は True。
config_source設定ソース。
is_sensitiveTrue if this configuration is sensitive.
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

DeleteTopics API (Key: 20):
Requests:

DeleteTopics Request (Version: 0) => [topic_names] timeout_ms 
  topic_names => STRING
  timeout_ms => INT32
フィールド 説明
topic_names削除するトピックの名前。
timeout_ms削除が完了するまで待つ時間の長さのミリ秒。

DeleteTopics Request (Version: 1) => [topic_names] timeout_ms 
  topic_names => STRING
  timeout_ms => INT32
フィールド 説明
topic_names削除するトピックの名前。
timeout_ms削除が完了するまで待つ時間の長さのミリ秒。

DeleteTopics Request (Version: 2) => [topic_names] timeout_ms 
  topic_names => STRING
  timeout_ms => INT32
フィールド 説明
topic_names削除するトピックの名前。
timeout_ms削除が完了するまで待つ時間の長さのミリ秒。

DeleteTopics Request (Version: 3) => [topic_names] timeout_ms 
  topic_names => STRING
  timeout_ms => INT32
フィールド 説明
topic_names削除するトピックの名前。
timeout_ms削除が完了するまで待つ時間の長さのミリ秒。

DeleteTopics Request (Version: 4) => [topic_names] timeout_ms TAG_BUFFER 
  topic_names => COMPACT_STRING
  timeout_ms => INT32
フィールド 説明
topic_names削除するトピックの名前。
timeout_ms削除が完了するまで待つ時間の長さのミリ秒。
_tagged_fieldsタグ付きのフィールド

DeleteTopics Request (Version: 5) => [topic_names] timeout_ms TAG_BUFFER 
  topic_names => COMPACT_STRING
  timeout_ms => INT32
フィールド 説明
topic_names削除するトピックの名前。
timeout_ms削除が完了するまで待つ時間の長さのミリ秒。
_tagged_fieldsタグ付きのフィールド

Responses:

DeleteTopics Response (Version: 0) => [responses] 
  responses => name error_code 
    name => STRING
    error_code => INT16
フィールド 説明
responses削除しようとした各トピックの結果。
名前トピック名。
error_code削除エラー。あるいは削除が成功した場合は0。

DeleteTopics Response (Version: 1) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => name error_code 
    name => STRING
    error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
responses削除しようとした各トピックの結果。
名前トピック名。
error_code削除エラー。あるいは削除が成功した場合は0。

DeleteTopics Response (Version: 2) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => name error_code 
    name => STRING
    error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
responses削除しようとした各トピックの結果。
名前トピック名。
error_code削除エラー。あるいは削除が成功した場合は0。

DeleteTopics Response (Version: 3) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => name error_code 
    name => STRING
    error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
responses削除しようとした各トピックの結果。
名前トピック名。
error_code削除エラー。あるいは削除が成功した場合は0。

DeleteTopics Response (Version: 4) => throttle_time_ms [responses] TAG_BUFFER 
  throttle_time_ms => INT32
  responses => name error_code TAG_BUFFER 
    name => COMPACT_STRING
    error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
responses削除しようとした各トピックの結果。
名前トピック名。
error_code削除エラー。あるいは削除が成功した場合は0。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

DeleteTopics Response (Version: 5) => throttle_time_ms [responses] TAG_BUFFER 
  throttle_time_ms => INT32
  responses => name error_code error_message TAG_BUFFER 
    name => COMPACT_STRING
    error_code => INT16
    error_message => COMPACT_NULLABLE_STRING
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
responses削除しようとした各トピックの結果。
名前トピック名。
error_code削除エラー。あるいは削除が成功した場合は0。
error_messageエラーメッセージ。あるいはエラーが無かった場合はnull。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

DeleteRecords API (Key: 21):
Requests:

DeleteRecords Request (Version: 0) => [topics] timeout_ms 
  topics => name [partitions] 
    name => STRING
    partitions => partition_index offset 
      partition_index => INT32
      offset => INT64
  timeout_ms => INT32
フィールド 説明
topicsレコードを削除しようとする各トピック。
名前トピック名。
partitionsレコードを削除しようとする各パーティション。
partition_indexパーティションのインデックス。
オフセット削除オフセット。
timeout_ms削除が完了するまでどれだけ待つかのミリ秒。

DeleteRecords Request (Version: 1) => [topics] timeout_ms 
  topics => name [partitions] 
    name => STRING
    partitions => partition_index offset 
      partition_index => INT32
      offset => INT64
  timeout_ms => INT32
フィールド 説明
topicsレコードを削除しようとする各トピック。
名前トピック名。
partitionsレコードを削除しようとする各パーティション。
partition_indexパーティションのインデックス。
オフセット削除オフセット。
timeout_ms削除が完了するまでどれだけ待つかのミリ秒。

DeleteRecords Request (Version: 2) => [topics] timeout_ms TAG_BUFFER 
  topics => name [partitions] TAG_BUFFER 
    name => COMPACT_STRING
    partitions => partition_index offset TAG_BUFFER 
      partition_index => INT32
      offset => INT64
  timeout_ms => INT32
フィールド 説明
topicsレコードを削除しようとする各トピック。
名前トピック名。
partitionsレコードを削除しようとする各パーティション。
partition_indexパーティションのインデックス。
オフセット削除オフセット。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド
timeout_ms削除が完了するまでどれだけ待つかのミリ秒。
_tagged_fieldsタグ付きのフィールド

Responses:

DeleteRecords Response (Version: 0) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index low_watermark error_code 
      partition_index => INT32
      low_watermark => INT64
      error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
topicsレコードを削除しようとする各トピック。
名前トピック名。
partitionsレコードを削除しようとする各パーティション。
partition_indexパーティションのインデックス。
low_watermarkThe partition low water mark.
error_code削除のエラーコード。あるいは削除が成功した場合は 0。

DeleteRecords Response (Version: 1) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index low_watermark error_code 
      partition_index => INT32
      low_watermark => INT64
      error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
topicsレコードを削除しようとする各トピック。
名前トピック名。
partitionsレコードを削除しようとする各パーティション。
partition_indexパーティションのインデックス。
low_watermarkThe partition low water mark.
error_code削除のエラーコード。あるいは削除が成功した場合は 0。

DeleteRecords Response (Version: 2) => throttle_time_ms [topics] TAG_BUFFER 
  throttle_time_ms => INT32
  topics => name [partitions] TAG_BUFFER 
    name => COMPACT_STRING
    partitions => partition_index low_watermark error_code TAG_BUFFER 
      partition_index => INT32
      low_watermark => INT64
      error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
topicsレコードを削除しようとする各トピック。
名前トピック名。
partitionsレコードを削除しようとする各パーティション。
partition_indexパーティションのインデックス。
low_watermarkThe partition low water mark.
error_code削除のエラーコード。あるいは削除が成功した場合は 0。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

InitProducerId API (Key: 22):
Requests:

InitProducerId Request (Version: 0) => transactional_id transaction_timeout_ms 
  transactional_id => NULLABLE_STRING
  transaction_timeout_ms => INT32
フィールド 説明
transactional_idトランザクション id。あるいはプロデューサがトランザクション的ではない場合 null。
transaction_timeout_msこのプロデューサによって送信される何もしていないトランザクションを中止する前に待つ時間のミリ秒。TransactionalIdが定義された場合のみ関係があります。

InitProducerId Request (Version: 1) => transactional_id transaction_timeout_ms 
  transactional_id => NULLABLE_STRING
  transaction_timeout_ms => INT32
フィールド 説明
transactional_idトランザクション id。あるいはプロデューサがトランザクション的ではない場合 null。
transaction_timeout_msこのプロデューサによって送信される何もしていないトランザクションを中止する前に待つ時間のミリ秒。TransactionalIdが定義された場合のみ関係があります。

InitProducerId Request (Version: 2) => transactional_id transaction_timeout_ms TAG_BUFFER 
  transactional_id => COMPACT_NULLABLE_STRING
  transaction_timeout_ms => INT32
フィールド 説明
transactional_idトランザクション id。あるいはプロデューサがトランザクション的ではない場合 null。
transaction_timeout_msこのプロデューサによって送信される何もしていないトランザクションを中止する前に待つ時間のミリ秒。TransactionalIdが定義された場合のみ関係があります。
_tagged_fieldsタグ付きのフィールド

InitProducerId Request (Version: 3) => transactional_id transaction_timeout_ms producer_id producer_epoch TAG_BUFFER 
  transactional_id => COMPACT_NULLABLE_STRING
  transaction_timeout_ms => INT32
  producer_id => INT64
  producer_epoch => INT16
フィールド 説明
transactional_idトランザクション id。あるいはプロデューサがトランザクション的ではない場合 null。
transaction_timeout_msこのプロデューサによって送信される何もしていないトランザクションを中止する前に待つ時間のミリ秒。TransactionalIdが定義された場合のみ関係があります。
producer_idプロデューサ id。これは、トランザクション id が有効期限後に再利用された場合に、リクエストを明確にするために使われます。
producer_epochプロデューサの現在のエポック。これは、ブローカーのプロデューサエポックに対してチェックされ、一致しない場合はリクエストはエラーを返します。
_tagged_fieldsタグ付きのフィールド

InitProducerId Request (Version: 4) => transactional_id transaction_timeout_ms producer_id producer_epoch TAG_BUFFER 
  transactional_id => COMPACT_NULLABLE_STRING
  transaction_timeout_ms => INT32
  producer_id => INT64
  producer_epoch => INT16
フィールド 説明
transactional_idトランザクション id。あるいはプロデューサがトランザクション的ではない場合 null。
transaction_timeout_msこのプロデューサによって送信される何もしていないトランザクションを中止する前に待つ時間のミリ秒。TransactionalIdが定義された場合のみ関係があります。
producer_idプロデューサ id。これは、トランザクション id が有効期限後に再利用された場合に、リクエストを明確にするために使われます。
producer_epochプロデューサの現在のエポック。これは、ブローカーのプロデューサエポックに対してチェックされ、一致しない場合はリクエストはエラーを返します。
_tagged_fieldsタグ付きのフィールド

Responses:

InitProducerId Response (Version: 0) => throttle_time_ms error_code producer_id producer_epoch 
  throttle_time_ms => INT32
  error_code => INT16
  producer_id => INT64
  producer_epoch => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。
producer_id現在のプロデューサ id。
producer_epochプロデューサ id に関連する現在のエポック。

InitProducerId Response (Version: 1) => throttle_time_ms error_code producer_id producer_epoch 
  throttle_time_ms => INT32
  error_code => INT16
  producer_id => INT64
  producer_epoch => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。
producer_id現在のプロデューサ id。
producer_epochプロデューサ id に関連する現在のエポック。

InitProducerId Response (Version: 2) => throttle_time_ms error_code producer_id producer_epoch TAG_BUFFER 
  throttle_time_ms => INT32
  error_code => INT16
  producer_id => INT64
  producer_epoch => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。
producer_id現在のプロデューサ id。
producer_epochプロデューサ id に関連する現在のエポック。
_tagged_fieldsタグ付きのフィールド

InitProducerId Response (Version: 3) => throttle_time_ms error_code producer_id producer_epoch TAG_BUFFER 
  throttle_time_ms => INT32
  error_code => INT16
  producer_id => INT64
  producer_epoch => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。
producer_id現在のプロデューサ id。
producer_epochプロデューサ id に関連する現在のエポック。
_tagged_fieldsタグ付きのフィールド

InitProducerId Response (Version: 4) => throttle_time_ms error_code producer_id producer_epoch TAG_BUFFER 
  throttle_time_ms => INT32
  error_code => INT16
  producer_id => INT64
  producer_epoch => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。
producer_id現在のプロデューサ id。
producer_epochプロデューサ id に関連する現在のエポック。
_tagged_fieldsタグ付きのフィールド

OffsetForLeaderEpoch API (Key: 23):
Requests:

OffsetForLeaderEpoch Request (Version: 0) => [topics] 
  topics => topic [partitions] 
    topic => STRING
    partitions => partition leader_epoch 
      partition => INT32
      leader_epoch => INT32
フィールド 説明
topicsエポックを取得するためのトピックの配列。
トピックトピック名
partitionsエポックを取得するためのパーティションの配列。
パーティショントピックのパーティションid
leader_epochオフセットを探すエポック。

OffsetForLeaderEpoch Request (Version: 1) => [topics] 
  topics => topic [partitions] 
    topic => STRING
    partitions => partition leader_epoch 
      partition => INT32
      leader_epoch => INT32
フィールド 説明
topicsエポックを取得するためのトピックの配列。
トピックトピック名
partitionsエポックを取得するためのパーティションの配列。
パーティショントピックのパーティションid
leader_epochオフセットを探すエポック。

OffsetForLeaderEpoch Request (Version: 2) => [topics] 
  topics => topic [partitions] 
    topic => STRING
    partitions => partition current_leader_epoch leader_epoch 
      partition => INT32
      current_leader_epoch => INT32
      leader_epoch => INT32
フィールド 説明
topicsエポックを取得するためのトピックの配列。
トピックトピック名
partitionsエポックを取得するためのパーティションの配列。
パーティショントピックのパーティションid
current_leader_epoch現在のリーダー エポックが提供されている場合、古いメタデータでコンシューマ/レプリカを区切るために使われます。クライアントによって提供されたエポックがブローカーに知られている現在のエポックより大きい場合、UNKNOWN_LEADER_EPOCH エラーコードが返されます。指定されたエポックが小さい場合、FENCED_LEADER_EPOCH エラーコードが返されます。
leader_epochオフセットを探すエポック。

OffsetForLeaderEpoch Request (Version: 3) => replica_id [topics] 
  replica_id => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => partition current_leader_epoch leader_epoch 
      partition => INT32
      current_leader_epoch => INT32
      leader_epoch => INT32
フィールド 説明
replica_idフォロワーのブローカー id。通常のコンシューマについては、-1を使ってください。
topicsエポックを取得するためのトピックの配列。
トピックトピック名
partitionsエポックを取得するためのパーティションの配列。
パーティショントピックのパーティションid
current_leader_epoch現在のリーダー エポックが提供されている場合、古いメタデータでコンシューマ/レプリカを区切るために使われます。クライアントによって提供されたエポックがブローカーに知られている現在のエポックより大きい場合、UNKNOWN_LEADER_EPOCH エラーコードが返されます。指定されたエポックが小さい場合、FENCED_LEADER_EPOCH エラーコードが返されます。
leader_epochオフセットを探すエポック。

Responses:

OffsetForLeaderEpoch Response (Version: 0) => [topics] 
  topics => topic [partitions] 
    topic => STRING
    partitions => error_code partition end_offset 
      error_code => INT16
      partition => INT32
      end_offset => INT64
フィールド 説明
topics要求されたパーティション リーダーのエポックのリーダー オフセットを持つトピックの配列。
トピックトピック名
partitionsパーティションによるオフセットの配列。
error_code応答エラーコード
パーティショントピックのパーティションid
end_offset最後のオフセット

OffsetForLeaderEpoch Response (Version: 1) => [topics] 
  topics => topic [partitions] 
    topic => STRING
    partitions => error_code partition leader_epoch end_offset 
      error_code => INT16
      partition => INT32
      leader_epoch => INT32
      end_offset => INT64
フィールド 説明
topics要求されたパーティション リーダーのエポックのリーダー オフセットを持つトピックの配列。
トピックトピック名
partitionsパーティションによるオフセットの配列。
error_code応答エラーコード
パーティショントピックのパーティションid
leader_epochリーダーのエポック
end_offset最後のオフセット

OffsetForLeaderEpoch Response (Version: 2) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => error_code partition leader_epoch end_offset 
      error_code => INT16
      partition => INT32
      leader_epoch => INT32
      end_offset => INT64
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間 (リクエストが何も割り当て量に違反しなかった場合は0)
topics要求されたパーティション リーダーのエポックのリーダー オフセットを持つトピックの配列。
トピックトピック名
partitionsパーティションによるオフセットの配列。
error_code応答エラーコード
パーティショントピックのパーティションid
leader_epochリーダーのエポック
end_offset最後のオフセット

OffsetForLeaderEpoch Response (Version: 3) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => topic [partitions] 
    topic => STRING
    partitions => error_code partition leader_epoch end_offset 
      error_code => INT16
      partition => INT32
      leader_epoch => INT32
      end_offset => INT64
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間 (リクエストが何も割り当て量に違反しなかった場合は0)
topics要求されたパーティション リーダーのエポックのリーダー オフセットを持つトピックの配列。
トピックトピック名
partitionsパーティションによるオフセットの配列。
error_code応答エラーコード
パーティショントピックのパーティションid
leader_epochリーダーのエポック
end_offset最後のオフセット

AddPartitionsToTxn API (Key: 24):
Requests:

AddPartitionsToTxn Request (Version: 0) => transactional_id producer_id producer_epoch [topics] 
  transactional_id => STRING
  producer_id => INT64
  producer_epoch => INT16
  topics => name [partitions] 
    name => STRING
    partitions => INT32
フィールド 説明
transactional_idトランザクションに対応するトランザクションid。
producer_idトランザクション id によって使われている現在のプロデューサid。
producer_epochプロデューサidに関連する現在のepoch。
topicsトランザクションに追加するパーティション。
名前トピックの名前。
partitionsThe partition indexes to add to the transaction

AddPartitionsToTxn Request (Version: 1) => transactional_id producer_id producer_epoch [topics] 
  transactional_id => STRING
  producer_id => INT64
  producer_epoch => INT16
  topics => name [partitions] 
    name => STRING
    partitions => INT32
フィールド 説明
transactional_idトランザクションに対応するトランザクションid。
producer_idトランザクション id によって使われている現在のプロデューサid。
producer_epochプロデューサidに関連する現在のepoch。
topicsトランザクションに追加するパーティション。
名前トピックの名前。
partitionsThe partition indexes to add to the transaction

AddPartitionsToTxn Request (Version: 2) => transactional_id producer_id producer_epoch [topics] 
  transactional_id => STRING
  producer_id => INT64
  producer_epoch => INT16
  topics => name [partitions] 
    name => STRING
    partitions => INT32
フィールド 説明
transactional_idトランザクションに対応するトランザクションid。
producer_idトランザクション id によって使われている現在のプロデューサid。
producer_epochプロデューサidに関連する現在のepoch。
topicsトランザクションに追加するパーティション。
名前トピックの名前。
partitionsThe partition indexes to add to the transaction

Responses:

AddPartitionsToTxn Response (Version: 0) => throttle_time_ms [results] 
  throttle_time_ms => INT32
  results => name [results] 
    name => STRING
    results => partition_index error_code 
      partition_index => INT32
      error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
results各トピックについての結果。
名前トピック名。
results各パーティションの結果
partition_indexパーティションのインデックス。
error_code応答エラーコード。

AddPartitionsToTxn Response (Version: 1) => throttle_time_ms [results] 
  throttle_time_ms => INT32
  results => name [results] 
    name => STRING
    results => partition_index error_code 
      partition_index => INT32
      error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
results各トピックについての結果。
名前トピック名。
results各パーティションの結果
partition_indexパーティションのインデックス。
error_code応答エラーコード。

AddPartitionsToTxn Response (Version: 2) => throttle_time_ms [results] 
  throttle_time_ms => INT32
  results => name [results] 
    name => STRING
    results => partition_index error_code 
      partition_index => INT32
      error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
results各トピックについての結果。
名前トピック名。
results各パーティションの結果
partition_indexパーティションのインデックス。
error_code応答エラーコード。

AddOffsetsToTxn API (Key: 25):
Requests:

AddOffsetsToTxn Request (Version: 0) => transactional_id producer_id producer_epoch group_id 
  transactional_id => STRING
  producer_id => INT64
  producer_epoch => INT16
  group_id => STRING
フィールド 説明
transactional_idトランザクションに対応するトランザクションid。
producer_idトランザクション id によって使われている現在のプロデューサid。
producer_epochプロデューサidに関連する現在のepoch。
group_idユニークなグループの識別子。

AddOffsetsToTxn Request (Version: 1) => transactional_id producer_id producer_epoch group_id 
  transactional_id => STRING
  producer_id => INT64
  producer_epoch => INT16
  group_id => STRING
フィールド 説明
transactional_idトランザクションに対応するトランザクションid。
producer_idトランザクション id によって使われている現在のプロデューサid。
producer_epochプロデューサidに関連する現在のepoch。
group_idユニークなグループの識別子。

AddOffsetsToTxn Request (Version: 2) => transactional_id producer_id producer_epoch group_id 
  transactional_id => STRING
  producer_id => INT64
  producer_epoch => INT16
  group_id => STRING
フィールド 説明
transactional_idトランザクションに対応するトランザクションid。
producer_idトランザクション id によって使われている現在のプロデューサid。
producer_epochプロデューサidに関連する現在のepoch。
group_idユニークなグループの識別子。

Responses:

AddOffsetsToTxn Response (Version: 0) => throttle_time_ms error_code 
  throttle_time_ms => INT32
  error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_code応答エラーコード。あるいはエラーが無かった場合は 0。

AddOffsetsToTxn Response (Version: 1) => throttle_time_ms error_code 
  throttle_time_ms => INT32
  error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_code応答エラーコード。あるいはエラーが無かった場合は 0。

AddOffsetsToTxn Response (Version: 2) => throttle_time_ms error_code 
  throttle_time_ms => INT32
  error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_code応答エラーコード。あるいはエラーが無かった場合は 0。

EndTxn API (Key: 26):
Requests:

EndTxn Request (Version: 0) => transactional_id producer_id producer_epoch committed 
  transactional_id => STRING
  producer_id => INT64
  producer_epoch => INT16
  committed => BOOLEAN
フィールド 説明
transactional_id終了するトランザクションの ID。
producer_idプロデューサ ID。
producer_epochプロデューサに関連する現在のエポック。
committedトランザクションがコミットされた場合は True。中止された場合は false。

EndTxn Request (Version: 1) => transactional_id producer_id producer_epoch committed 
  transactional_id => STRING
  producer_id => INT64
  producer_epoch => INT16
  committed => BOOLEAN
フィールド 説明
transactional_id終了するトランザクションの ID。
producer_idプロデューサ ID。
producer_epochプロデューサに関連する現在のエポック。
committedトランザクションがコミットされた場合は True。中止された場合は false。

EndTxn Request (Version: 2) => transactional_id producer_id producer_epoch committed 
  transactional_id => STRING
  producer_id => INT64
  producer_epoch => INT16
  committed => BOOLEAN
フィールド 説明
transactional_id終了するトランザクションの ID。
producer_idプロデューサ ID。
producer_epochプロデューサに関連する現在のエポック。
committedトランザクションがコミットされた場合は True。中止された場合は false。

Responses:

EndTxn Response (Version: 0) => throttle_time_ms error_code 
  throttle_time_ms => INT32
  error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。

EndTxn Response (Version: 1) => throttle_time_ms error_code 
  throttle_time_ms => INT32
  error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。

EndTxn Response (Version: 2) => throttle_time_ms error_code 
  throttle_time_ms => INT32
  error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。

WriteTxnMarkers API (Key: 27):
Requests:

WriteTxnMarkers Request (Version: 0) => [markers] 
  markers => producer_id producer_epoch transaction_result [topics] coordinator_epoch 
    producer_id => INT64
    producer_epoch => INT16
    transaction_result => BOOLEAN
    topics => name [partition_indexes] 
      name => STRING
      partition_indexes => INT32
    coordinator_epoch => INT32
フィールド 説明
markers書かれるトランザクション マーカー。
producer_id現在のプロデューサ ID。
producer_epochプロデューサ ID に関連する現在のエポック。
transaction_resultパーティションに書き込むトランザクションの結果 (false = ABORT, true = COMMIT).
topicsトランザクションマーカーを書き込む各トピック。
名前トピック名。
partition_indexesトランザクションマーカーを書き込むパーティションのインデックス。
coordinator_epochこのトランザクション コーディネータによってホストされるトランザクション状態パーティションに関連するepoch

Responses:

WriteTxnMarkers Response (Version: 0) => [markers] 
  markers => producer_id [topics] 
    producer_id => INT64
    topics => name [partitions] 
      name => STRING
      partitions => partition_index error_code 
        partition_index => INT32
        error_code => INT16
フィールド 説明
markersマーカーの書き込み結果。
producer_idトランザクション ID によって使われる現在のプロデューサ ID。
topicsトピックの結果。
名前トピック名。
partitionsパーティションの結果。
partition_indexパーティションのインデックス。
error_codeエラーコード。あるいはエラーが無かった場合は0。

TxnOffsetCommit API (Key: 28):
Requests:

TxnOffsetCommit Request (Version: 0) => transactional_id group_id producer_id producer_epoch [topics] 
  transactional_id => STRING
  group_id => STRING
  producer_id => INT64
  producer_epoch => INT16
  topics => name [partitions] 
    name => STRING
    partitions => partition_index committed_offset committed_metadata 
      partition_index => INT32
      committed_offset => INT64
      committed_metadata => NULLABLE_STRING
フィールド 説明
transactional_idトランザクションの ID。
group_idグループの ID。
producer_idトランザクション ID によって使われる現在のプロデューサ ID。
producer_epochプロデューサ ID に関連する現在のエポック。
topicsオフセットをコミットしようとする各トピック。
名前トピック名。
partitionsオフセットをコミットしようとするトピック内のパーティション。
partition_indexトピック内のパーティションのインデックス。
committed_offsetコミットされるメッセージのオフセット。
committed_metadataクライアントが維持したい関連メタデータ。

TxnOffsetCommit Request (Version: 1) => transactional_id group_id producer_id producer_epoch [topics] 
  transactional_id => STRING
  group_id => STRING
  producer_id => INT64
  producer_epoch => INT16
  topics => name [partitions] 
    name => STRING
    partitions => partition_index committed_offset committed_metadata 
      partition_index => INT32
      committed_offset => INT64
      committed_metadata => NULLABLE_STRING
フィールド 説明
transactional_idトランザクションの ID。
group_idグループの ID。
producer_idトランザクション ID によって使われる現在のプロデューサ ID。
producer_epochプロデューサ ID に関連する現在のエポック。
topicsオフセットをコミットしようとする各トピック。
名前トピック名。
partitionsオフセットをコミットしようとするトピック内のパーティション。
partition_indexトピック内のパーティションのインデックス。
committed_offsetコミットされるメッセージのオフセット。
committed_metadataクライアントが維持したい関連メタデータ。

TxnOffsetCommit Request (Version: 2) => transactional_id group_id producer_id producer_epoch [topics] 
  transactional_id => STRING
  group_id => STRING
  producer_id => INT64
  producer_epoch => INT16
  topics => name [partitions] 
    name => STRING
    partitions => partition_index committed_offset committed_leader_epoch committed_metadata 
      partition_index => INT32
      committed_offset => INT64
      committed_leader_epoch => INT32
      committed_metadata => NULLABLE_STRING
フィールド 説明
transactional_idトランザクションの ID。
group_idグループの ID。
producer_idトランザクション ID によって使われる現在のプロデューサ ID。
producer_epochプロデューサ ID に関連する現在のエポック。
topicsオフセットをコミットしようとする各トピック。
名前トピック名。
partitionsオフセットをコミットしようとするトピック内のパーティション。
partition_indexトピック内のパーティションのインデックス。
committed_offsetコミットされるメッセージのオフセット。
committed_leader_epoch最後に消費されたレコードのリーダーエポック。
committed_metadataクライアントが維持したい関連メタデータ。

TxnOffsetCommit Request (Version: 3) => transactional_id group_id producer_id producer_epoch generation_id member_id group_instance_id [topics] TAG_BUFFER 
  transactional_id => COMPACT_STRING
  group_id => COMPACT_STRING
  producer_id => INT64
  producer_epoch => INT16
  generation_id => INT32
  member_id => COMPACT_STRING
  group_instance_id => COMPACT_NULLABLE_STRING
  topics => name [partitions] TAG_BUFFER 
    name => COMPACT_STRING
    partitions => partition_index committed_offset committed_leader_epoch committed_metadata TAG_BUFFER 
      partition_index => INT32
      committed_offset => INT64
      committed_leader_epoch => INT32
      committed_metadata => COMPACT_NULLABLE_STRING
フィールド 説明
transactional_idトランザクションの ID。
group_idグループの ID。
producer_idトランザクション ID によって使われる現在のプロデューサ ID。
producer_epochプロデューサ ID に関連する現在のエポック。
generation_idコンシューマの世代。
member_idグループ コーディネータによって割り当てられたメンバーID。
group_instance_idエンドユーザによって提供されるコンシューマ インスタンスの一意の識別子。
topicsオフセットをコミットしようとする各トピック。
名前トピック名。
partitionsオフセットをコミットしようとするトピック内のパーティション。
partition_indexトピック内のパーティションのインデックス。
committed_offsetコミットされるメッセージのオフセット。
committed_leader_epoch最後に消費されたレコードのリーダーエポック。
committed_metadataクライアントが維持したい関連メタデータ。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

Responses:

TxnOffsetCommit Response (Version: 0) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index error_code 
      partition_index => INT32
      error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
topics各トピックについての応答。
名前トピック名。
partitionsトピック内の各パーティションについての応答。
partition_indexパーティションのインデックス。
error_codeエラーコード。あるいはエラーが無かった場合は0。

TxnOffsetCommit Response (Version: 1) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index error_code 
      partition_index => INT32
      error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
topics各トピックについての応答。
名前トピック名。
partitionsトピック内の各パーティションについての応答。
partition_indexパーティションのインデックス。
error_codeエラーコード。あるいはエラーが無かった場合は0。

TxnOffsetCommit Response (Version: 2) => throttle_time_ms [topics] 
  throttle_time_ms => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index error_code 
      partition_index => INT32
      error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
topics各トピックについての応答。
名前トピック名。
partitionsトピック内の各パーティションについての応答。
partition_indexパーティションのインデックス。
error_codeエラーコード。あるいはエラーが無かった場合は0。

TxnOffsetCommit Response (Version: 3) => throttle_time_ms [topics] TAG_BUFFER 
  throttle_time_ms => INT32
  topics => name [partitions] TAG_BUFFER 
    name => COMPACT_STRING
    partitions => partition_index error_code TAG_BUFFER 
      partition_index => INT32
      error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
topics各トピックについての応答。
名前トピック名。
partitionsトピック内の各パーティションについての応答。
partition_indexパーティションのインデックス。
error_codeエラーコード。あるいはエラーが無かった場合は0。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

DescribeAcls API (Key: 29):
Requests:

DescribeAcls Request (Version: 0) => resource_type_filter resource_name_filter principal_filter host_filter operation permission_type 
  resource_type_filter => INT8
  resource_name_filter => NULLABLE_STRING
  principal_filter => NULLABLE_STRING
  host_filter => NULLABLE_STRING
  operation => INT8
  permission_type => INT8
フィールド 説明
resource_type_filterリソース型。
resource_name_filterリソース名。または全てのリソース名に一致させるための null。
principal_filter一致するプリンシパル。あるいは全てのプリンシパルに一致させるための null。
host_filter一致するホスト。あるいは全てのホストに一致させるための null。
operation一致する操作
permission_type一致するパーミッション型。

DescribeAcls Request (Version: 1) => resource_type_filter resource_name_filter pattern_type_filter principal_filter host_filter operation permission_type 
  resource_type_filter => INT8
  resource_name_filter => NULLABLE_STRING
  pattern_type_filter => INT8
  principal_filter => NULLABLE_STRING
  host_filter => NULLABLE_STRING
  operation => INT8
  permission_type => INT8
フィールド 説明
resource_type_filterリソース型。
resource_name_filterリソース名。または全てのリソース名に一致させるための null。
pattern_type_filter一致するリソースパターン。
principal_filter一致するプリンシパル。あるいは全てのプリンシパルに一致させるための null。
host_filter一致するホスト。あるいは全てのホストに一致させるための null。
operation一致する操作
permission_type一致するパーミッション型。

DescribeAcls Request (Version: 2) => resource_type_filter resource_name_filter pattern_type_filter principal_filter host_filter operation permission_type TAG_BUFFER 
  resource_type_filter => INT8
  resource_name_filter => COMPACT_NULLABLE_STRING
  pattern_type_filter => INT8
  principal_filter => COMPACT_NULLABLE_STRING
  host_filter => COMPACT_NULLABLE_STRING
  operation => INT8
  permission_type => INT8
フィールド 説明
resource_type_filterリソース型。
resource_name_filterリソース名。または全てのリソース名に一致させるための null。
pattern_type_filter一致するリソースパターン。
principal_filter一致するプリンシパル。あるいは全てのプリンシパルに一致させるための null。
host_filter一致するホスト。あるいは全てのホストに一致させるための null。
operation一致する操作
permission_type一致するパーミッション型。
_tagged_fieldsタグ付きのフィールド

Responses:

DescribeAcls Response (Version: 0) => throttle_time_ms error_code error_message [resources] 
  throttle_time_ms => INT32
  error_code => INT16
  error_message => NULLABLE_STRING
  resources => resource_type resource_name [acls] 
    resource_type => INT8
    resource_name => STRING
    acls => principal host operation permission_type 
      principal => STRING
      host => STRING
      operation => INT8
      permission_type => INT8
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。
error_messageエラーメッセージ。あるいはエラーが無かった場合はnull。
resourcesACL で参照される各リソース。
resource_typeリソース型。
resource_nameリソース名。
aclsACL。
principalACL プリンシパル。
hostACL ホスト。
operationACL 操作。
permission_typeACL パーミッション型。

DescribeAcls Response (Version: 1) => throttle_time_ms error_code error_message [resources] 
  throttle_time_ms => INT32
  error_code => INT16
  error_message => NULLABLE_STRING
  resources => resource_type resource_name pattern_type [acls] 
    resource_type => INT8
    resource_name => STRING
    pattern_type => INT8
    acls => principal host operation permission_type 
      principal => STRING
      host => STRING
      operation => INT8
      permission_type => INT8
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。
error_messageエラーメッセージ。あるいはエラーが無かった場合はnull。
resourcesACL で参照される各リソース。
resource_typeリソース型。
resource_nameリソース名。
pattern_typeリソースパターン型。
aclsACL。
principalACL プリンシパル。
hostACL ホスト。
operationACL 操作。
permission_typeACL パーミッション型。

DescribeAcls Response (Version: 2) => throttle_time_ms error_code error_message [resources] TAG_BUFFER 
  throttle_time_ms => INT32
  error_code => INT16
  error_message => COMPACT_NULLABLE_STRING
  resources => resource_type resource_name pattern_type [acls] TAG_BUFFER 
    resource_type => INT8
    resource_name => COMPACT_STRING
    pattern_type => INT8
    acls => principal host operation permission_type TAG_BUFFER 
      principal => COMPACT_STRING
      host => COMPACT_STRING
      operation => INT8
      permission_type => INT8
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeエラーコード。あるいはエラーが無かった場合は0。
error_messageエラーメッセージ。あるいはエラーが無かった場合はnull。
resourcesACL で参照される各リソース。
resource_typeリソース型。
resource_nameリソース名。
pattern_typeリソースパターン型。
aclsACL。
principalACL プリンシパル。
hostACL ホスト。
operationACL 操作。
permission_typeACL パーミッション型。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

CreateAcls API (Key: 30):
Requests:

CreateAcls Request (Version: 0) => [creations] 
  creations => resource_type resource_name principal host operation permission_type 
    resource_type => INT8
    resource_name => STRING
    principal => STRING
    host => STRING
    operation => INT8
    permission_type => INT8
フィールド 説明
creations作成したい ACL。
resource_typeリソースの型。
resource_nameACL のリソース名。
principalACL のプリンシパル。
hostACL のホスト名。
operationACL のオペレーション型 (read, write など)。
permission_typeACL のパーミッション型 (allow, deny など)。

CreateAcls Request (Version: 1) => [creations] 
  creations => resource_type resource_name resource_pattern_type principal host operation permission_type 
    resource_type => INT8
    resource_name => STRING
    resource_pattern_type => INT8
    principal => STRING
    host => STRING
    operation => INT8
    permission_type => INT8
フィールド 説明
creations作成したい ACL。
resource_typeリソースの型。
resource_nameACL のリソース名。
resource_pattern_typeACL のパターン型。
principalACL のプリンシパル。
hostACL のホスト名。
operationACL のオペレーション型 (read, write など)。
permission_typeACL のパーミッション型 (allow, deny など)。

CreateAcls Request (Version: 2) => [creations] TAG_BUFFER 
  creations => resource_type resource_name resource_pattern_type principal host operation permission_type TAG_BUFFER 
    resource_type => INT8
    resource_name => COMPACT_STRING
    resource_pattern_type => INT8
    principal => COMPACT_STRING
    host => COMPACT_STRING
    operation => INT8
    permission_type => INT8
フィールド 説明
creations作成したい ACL。
resource_typeリソースの型。
resource_nameACL のリソース名。
resource_pattern_typeACL のパターン型。
principalACL のプリンシパル。
hostACL のホスト名。
operationACL のオペレーション型 (read, write など)。
permission_typeACL のパーミッション型 (allow, deny など)。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

Responses:

CreateAcls Response (Version: 0) => throttle_time_ms [results] 
  throttle_time_ms => INT32
  results => error_code error_message 
    error_code => INT16
    error_message => NULLABLE_STRING
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
results各 ACL 作成の結果。
error_code結果エラー。あるいはエラーが無かった場合は0。
error_messageエラーメッセージ。あるいはエラーが無かった場合はnull。

CreateAcls Response (Version: 1) => throttle_time_ms [results] 
  throttle_time_ms => INT32
  results => error_code error_message 
    error_code => INT16
    error_message => NULLABLE_STRING
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
results各 ACL 作成の結果。
error_code結果エラー。あるいはエラーが無かった場合は0。
error_messageエラーメッセージ。あるいはエラーが無かった場合はnull。

CreateAcls Response (Version: 2) => throttle_time_ms [results] TAG_BUFFER 
  throttle_time_ms => INT32
  results => error_code error_message TAG_BUFFER 
    error_code => INT16
    error_message => COMPACT_NULLABLE_STRING
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
results各 ACL 作成の結果。
error_code結果エラー。あるいはエラーが無かった場合は0。
error_messageエラーメッセージ。あるいはエラーが無かった場合はnull。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

DeleteAcls API (Key: 31):
Requests:

DeleteAcls Request (Version: 0) => [filters] 
  filters => resource_type_filter resource_name_filter principal_filter host_filter operation permission_type 
    resource_type_filter => INT8
    resource_name_filter => NULLABLE_STRING
    principal_filter => NULLABLE_STRING
    host_filter => NULLABLE_STRING
    operation => INT8
    permission_type => INT8
フィールド 説明
filtersACL を削除する場合に使うフィルタ。
resource_type_filterリソース型。
resource_name_filterリソース名。
principal_filterプリンシパルフィルタ。あるいは全てのプリンシパルを受け付ける場合は null。
host_filterホストフィルタ。あるいは全てのホストを受け付ける場合は null。
operationACL 操作。
permission_typeパーミッション型。

DeleteAcls Request (Version: 1) => [filters] 
  filters => resource_type_filter resource_name_filter pattern_type_filter principal_filter host_filter operation permission_type 
    resource_type_filter => INT8
    resource_name_filter => NULLABLE_STRING
    pattern_type_filter => INT8
    principal_filter => NULLABLE_STRING
    host_filter => NULLABLE_STRING
    operation => INT8
    permission_type => INT8
フィールド 説明
filtersACL を削除する場合に使うフィルタ。
resource_type_filterリソース型。
resource_name_filterリソース名。
pattern_type_filterパターン型。
principal_filterプリンシパルフィルタ。あるいは全てのプリンシパルを受け付ける場合は null。
host_filterホストフィルタ。あるいは全てのホストを受け付ける場合は null。
operationACL 操作。
permission_typeパーミッション型。

DeleteAcls Request (Version: 2) => [filters] TAG_BUFFER 
  filters => resource_type_filter resource_name_filter pattern_type_filter principal_filter host_filter operation permission_type TAG_BUFFER 
    resource_type_filter => INT8
    resource_name_filter => COMPACT_NULLABLE_STRING
    pattern_type_filter => INT8
    principal_filter => COMPACT_NULLABLE_STRING
    host_filter => COMPACT_NULLABLE_STRING
    operation => INT8
    permission_type => INT8
フィールド 説明
filtersACL を削除する場合に使うフィルタ。
resource_type_filterリソース型。
resource_name_filterリソース名。
pattern_type_filterパターン型。
principal_filterプリンシパルフィルタ。あるいは全てのプリンシパルを受け付ける場合は null。
host_filterホストフィルタ。あるいは全てのホストを受け付ける場合は null。
operationACL 操作。
permission_typeパーミッション型。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

Responses:

DeleteAcls Response (Version: 0) => throttle_time_ms [filter_results] 
  throttle_time_ms => INT32
  filter_results => error_code error_message [matching_acls] 
    error_code => INT16
    error_message => NULLABLE_STRING
    matching_acls => error_code error_message resource_type resource_name principal host operation permission_type 
      error_code => INT16
      error_message => NULLABLE_STRING
      resource_type => INT8
      resource_name => STRING
      principal => STRING
      host => STRING
      operation => INT8
      permission_type => INT8
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
filter_results各フィルタの結果。
error_codeエラーコード。あるいはフィルタが成功した場合は 0。
error_messageエラーメッセージ。あるいはフィルタが成功した場合は null。
matching_aclsこのフィルタに一致する ACL。
error_code削除のエラーコード。あるいは削除が成功した場合は 0。
error_message削除エラーメッセージ。あるいは削除が成功した場合は null。
resource_typeACL リソース型。
resource_nameACL リソース名。
principalACL プリンシパル。
hostACL ホスト。
operationACL 操作。
permission_typeACL パーミッション型。

DeleteAcls Response (Version: 1) => throttle_time_ms [filter_results] 
  throttle_time_ms => INT32
  filter_results => error_code error_message [matching_acls] 
    error_code => INT16
    error_message => NULLABLE_STRING
    matching_acls => error_code error_message resource_type resource_name pattern_type principal host operation permission_type 
      error_code => INT16
      error_message => NULLABLE_STRING
      resource_type => INT8
      resource_name => STRING
      pattern_type => INT8
      principal => STRING
      host => STRING
      operation => INT8
      permission_type => INT8
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
filter_results各フィルタの結果。
error_codeエラーコード。あるいはフィルタが成功した場合は 0。
error_messageエラーメッセージ。あるいはフィルタが成功した場合は null。
matching_aclsこのフィルタに一致する ACL。
error_code削除のエラーコード。あるいは削除が成功した場合は 0。
error_message削除エラーメッセージ。あるいは削除が成功した場合は null。
resource_typeACL リソース型。
resource_nameACL リソース名。
pattern_typeACL リソースパターン型。
principalACL プリンシパル。
hostACL ホスト。
operationACL 操作。
permission_typeACL パーミッション型。

DeleteAcls Response (Version: 2) => throttle_time_ms [filter_results] TAG_BUFFER 
  throttle_time_ms => INT32
  filter_results => error_code error_message [matching_acls] TAG_BUFFER 
    error_code => INT16
    error_message => COMPACT_NULLABLE_STRING
    matching_acls => error_code error_message resource_type resource_name pattern_type principal host operation permission_type TAG_BUFFER 
      error_code => INT16
      error_message => COMPACT_NULLABLE_STRING
      resource_type => INT8
      resource_name => COMPACT_STRING
      pattern_type => INT8
      principal => COMPACT_STRING
      host => COMPACT_STRING
      operation => INT8
      permission_type => INT8
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
filter_results各フィルタの結果。
error_codeエラーコード。あるいはフィルタが成功した場合は 0。
error_messageエラーメッセージ。あるいはフィルタが成功した場合は null。
matching_aclsこのフィルタに一致する ACL。
error_code削除のエラーコード。あるいは削除が成功した場合は 0。
error_message削除エラーメッセージ。あるいは削除が成功した場合は null。
resource_typeACL リソース型。
resource_nameACL リソース名。
pattern_typeACL リソースパターン型。
principalACL プリンシパル。
hostACL ホスト。
operationACL 操作。
permission_typeACL パーミッション型。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

DescribeConfigs API (Key: 32):
Requests:

DescribeConfigs Request (Version: 0) => [resources] 
  resources => resource_type resource_name [configuration_keys] 
    resource_type => INT8
    resource_name => STRING
    configuration_keys => STRING
フィールド 説明
resourcesThe resources whose configurations we want to describe.
resource_typeリソース型。
resource_nameリソース名。
configuration_keysThe configuration keys to list, or null to list all configuration keys.

DescribeConfigs Request (Version: 1) => [resources] include_synonyms 
  resources => resource_type resource_name [configuration_keys] 
    resource_type => INT8
    resource_name => STRING
    configuration_keys => STRING
  include_synonyms => BOOLEAN
フィールド 説明
resourcesThe resources whose configurations we want to describe.
resource_typeリソース型。
resource_nameリソース名。
configuration_keysThe configuration keys to list, or null to list all configuration keys.
include_synonymsTrue if we should include all synonyms.

DescribeConfigs Request (Version: 2) => [resources] include_synonyms 
  resources => resource_type resource_name [configuration_keys] 
    resource_type => INT8
    resource_name => STRING
    configuration_keys => STRING
  include_synonyms => BOOLEAN
フィールド 説明
resourcesThe resources whose configurations we want to describe.
resource_typeリソース型。
resource_nameリソース名。
configuration_keysThe configuration keys to list, or null to list all configuration keys.
include_synonymsTrue if we should include all synonyms.

DescribeConfigs Request (Version: 3) => [resources] include_synonyms include_documentation 
  resources => resource_type resource_name [configuration_keys] 
    resource_type => INT8
    resource_name => STRING
    configuration_keys => STRING
  include_synonyms => BOOLEAN
  include_documentation => BOOLEAN
フィールド 説明
resourcesThe resources whose configurations we want to describe.
resource_typeリソース型。
resource_nameリソース名。
configuration_keysThe configuration keys to list, or null to list all configuration keys.
include_synonymsTrue if we should include all synonyms.
include_documentationTrue if we should include configuration documentation.

Responses:

DescribeConfigs Response (Version: 0) => throttle_time_ms [results] 
  throttle_time_ms => INT32
  results => error_code error_message resource_type resource_name [configs] 
    error_code => INT16
    error_message => NULLABLE_STRING
    resource_type => INT8
    resource_name => STRING
    configs => name value read_only is_default is_sensitive 
      name => STRING
      value => NULLABLE_STRING
      read_only => BOOLEAN
      is_default => BOOLEAN
      is_sensitive => BOOLEAN
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
resultsThe results for each resource.
error_codeThe error code, or 0 if we were able to successfully describe the configurations.
error_messageThe error message, or null if we were able to successfully describe the configurations.
resource_typeリソース型。
resource_nameリソース名。
configsEach listed configuration.
名前設定名。
設定値。
read_only設定が読み込み専用の場合は True。
is_defaultTrue if the configuration is not set.
is_sensitiveTrue if this configuration is sensitive.

DescribeConfigs Response (Version: 1) => throttle_time_ms [results] 
  throttle_time_ms => INT32
  results => error_code error_message resource_type resource_name [configs] 
    error_code => INT16
    error_message => NULLABLE_STRING
    resource_type => INT8
    resource_name => STRING
    configs => name value read_only config_source is_sensitive [synonyms] 
      name => STRING
      value => NULLABLE_STRING
      read_only => BOOLEAN
      config_source => INT8
      is_sensitive => BOOLEAN
      synonyms => name value source 
        name => STRING
        value => NULLABLE_STRING
        source => INT8
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
resultsThe results for each resource.
error_codeThe error code, or 0 if we were able to successfully describe the configurations.
error_messageThe error message, or null if we were able to successfully describe the configurations.
resource_typeリソース型。
resource_nameリソース名。
configsEach listed configuration.
名前設定名。
設定値。
read_only設定が読み込み専用の場合は True。
config_source設定ソース。
is_sensitiveTrue if this configuration is sensitive.
synonymsThe synonyms for this configuration key.
名前The synonym name.
The synonym value.
sourceThe synonym source.

DescribeConfigs Response (Version: 2) => throttle_time_ms [results] 
  throttle_time_ms => INT32
  results => error_code error_message resource_type resource_name [configs] 
    error_code => INT16
    error_message => NULLABLE_STRING
    resource_type => INT8
    resource_name => STRING
    configs => name value read_only config_source is_sensitive [synonyms] 
      name => STRING
      value => NULLABLE_STRING
      read_only => BOOLEAN
      config_source => INT8
      is_sensitive => BOOLEAN
      synonyms => name value source 
        name => STRING
        value => NULLABLE_STRING
        source => INT8
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
resultsThe results for each resource.
error_codeThe error code, or 0 if we were able to successfully describe the configurations.
error_messageThe error message, or null if we were able to successfully describe the configurations.
resource_typeリソース型。
resource_nameリソース名。
configsEach listed configuration.
名前設定名。
設定値。
read_only設定が読み込み専用の場合は True。
config_source設定ソース。
is_sensitiveTrue if this configuration is sensitive.
synonymsThe synonyms for this configuration key.
名前The synonym name.
The synonym value.
sourceThe synonym source.

DescribeConfigs Response (Version: 3) => throttle_time_ms [results] 
  throttle_time_ms => INT32
  results => error_code error_message resource_type resource_name [configs] 
    error_code => INT16
    error_message => NULLABLE_STRING
    resource_type => INT8
    resource_name => STRING
    configs => name value read_only config_source is_sensitive [synonyms] config_type documentation 
      name => STRING
      value => NULLABLE_STRING
      read_only => BOOLEAN
      config_source => INT8
      is_sensitive => BOOLEAN
      synonyms => name value source 
        name => STRING
        value => NULLABLE_STRING
        source => INT8
      config_type => INT8
      documentation => NULLABLE_STRING
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
resultsThe results for each resource.
error_codeThe error code, or 0 if we were able to successfully describe the configurations.
error_messageThe error message, or null if we were able to successfully describe the configurations.
resource_typeリソース型。
resource_nameリソース名。
configsEach listed configuration.
名前設定名。
設定値。
read_only設定が読み込み専用の場合は True。
config_source設定ソース。
is_sensitiveTrue if this configuration is sensitive.
synonymsThe synonyms for this configuration key.
名前The synonym name.
The synonym value.
sourceThe synonym source.
config_typeThe configuration data type. Type can be one of the following values - BOOLEAN, STRING, INT, SHORT, LONG, DOUBLE, LIST, CLASS, PASSWORD
ドキュメントThe configuration documentation.

AlterConfigs API (Key: 33):
Requests:

AlterConfigs Request (Version: 0) => [resources] validate_only 
  resources => resource_type resource_name [configs] 
    resource_type => INT8
    resource_name => STRING
    configs => name value 
      name => STRING
      value => NULLABLE_STRING
  validate_only => BOOLEAN
フィールド 説明
resources各リソースの更新。
resource_typeリソース型。
resource_nameリソース名。
configs設定。
名前設定キー名。
設定キーに設定する値。
validate_onlyリクエストを検証する必要があるが、設定を変更しない場合は true。

AlterConfigs Request (Version: 1) => [resources] validate_only 
  resources => resource_type resource_name [configs] 
    resource_type => INT8
    resource_name => STRING
    configs => name value 
      name => STRING
      value => NULLABLE_STRING
  validate_only => BOOLEAN
フィールド 説明
resources各リソースの更新。
resource_typeリソース型。
resource_nameリソース名。
configs設定。
名前設定キー名。
設定キーに設定する値。
validate_onlyリクエストを検証する必要があるが、設定を変更しない場合は true。

Responses:

AlterConfigs Response (Version: 0) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => error_code error_message resource_type resource_name 
    error_code => INT16
    error_message => NULLABLE_STRING
    resource_type => INT8
    resource_name => STRING
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
responses各リソースについての応答。
error_codeリソース エラーコード。
error_messageリソース エラー メッセージ。あるいはエラーが無かった場合はnull。
resource_typeリソース型。
resource_nameリソース名。

AlterConfigs Response (Version: 1) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => error_code error_message resource_type resource_name 
    error_code => INT16
    error_message => NULLABLE_STRING
    resource_type => INT8
    resource_name => STRING
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
responses各リソースについての応答。
error_codeリソース エラーコード。
error_messageリソース エラー メッセージ。あるいはエラーが無かった場合はnull。
resource_typeリソース型。
resource_nameリソース名。

AlterReplicaLogDirs API (Key: 34):
Requests:

AlterReplicaLogDirs Request (Version: 0) => [dirs] 
  dirs => path [topics] 
    path => STRING
    topics => name [partitions] 
      name => STRING
      partitions => INT32
フィールド 説明
dirsThe alterations to make for each directory.
pathThe absolute directory path.
topicsThe topics to add to the directory.
名前トピック名。
partitionsパーティションのインデックス。

AlterReplicaLogDirs Request (Version: 1) => [dirs] 
  dirs => path [topics] 
    path => STRING
    topics => name [partitions] 
      name => STRING
      partitions => INT32
フィールド 説明
dirsThe alterations to make for each directory.
pathThe absolute directory path.
topicsThe topics to add to the directory.
名前トピック名。
partitionsパーティションのインデックス。

Responses:

AlterReplicaLogDirs Response (Version: 0) => throttle_time_ms [results] 
  throttle_time_ms => INT32
  results => topic_name [partitions] 
    topic_name => STRING
    partitions => partition_index error_code 
      partition_index => INT32
      error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
results各トピックについての結果。
topic_nameトピックの名前。
partitionsThe results for each partition.
partition_indexパーティションのインデックス。
error_codeエラーコード。あるいはエラーが無かった場合は0。

AlterReplicaLogDirs Response (Version: 1) => throttle_time_ms [results] 
  throttle_time_ms => INT32
  results => topic_name [partitions] 
    topic_name => STRING
    partitions => partition_index error_code 
      partition_index => INT32
      error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
results各トピックについての結果。
topic_nameトピックの名前。
partitionsThe results for each partition.
partition_indexパーティションのインデックス。
error_codeエラーコード。あるいはエラーが無かった場合は0。

DescribeLogDirs API (Key: 35):
Requests:

DescribeLogDirs Request (Version: 0) => [topics] 
  topics => topic [partition_index] 
    topic => STRING
    partition_index => INT32
フィールド 説明
topicsログディレクトリを説明する各トピック。あるいは全てのトピックに対して null。
トピックトピック名。
partition_indexパーティションインデックス。

DescribeLogDirs Request (Version: 1) => [topics] 
  topics => topic [partition_index] 
    topic => STRING
    partition_index => INT32
フィールド 説明
topicsログディレクトリを説明する各トピック。あるいは全てのトピックに対して null。
トピックトピック名。
partition_indexパーティションインデックス。

DescribeLogDirs Request (Version: 2) => [topics] TAG_BUFFER 
  topics => topic [partition_index] TAG_BUFFER 
    topic => COMPACT_STRING
    partition_index => INT32
フィールド 説明
topicsログディレクトリを説明する各トピック。あるいは全てのトピックに対して null。
トピックトピック名。
partition_indexパーティションインデックス。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

Responses:

DescribeLogDirs Response (Version: 0) => throttle_time_ms [results] 
  throttle_time_ms => INT32
  results => error_code log_dir [topics] 
    error_code => INT16
    log_dir => STRING
    topics => name [partitions] 
      name => STRING
      partitions => partition_index partition_size offset_lag is_future_key 
        partition_index => INT32
        partition_size => INT64
        offset_lag => INT64
        is_future_key => BOOLEAN
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
resultsThe log directories.
error_codeエラーコード。あるいはエラーが無かった場合は0。
log_dirログディレクトリの絶対パス。
topics各トピック。
名前トピック名。
partitions
partition_indexパーティションのインデックス。
partition_sizeこのパーティション内のログセグメントのサイズのバイト。
offset_lagログのLEOに関連する遅延。(パーティションについて現在のログであれば)パーティションのHW、あるいは(パーティションについての未来のログであれば)現在のレプリカのLEO
is_future_keyこのログが AlterReplicaLogDirsRequest によって生成され、将来レプリカの現在のログに置き換えられる場合true。

DescribeLogDirs Response (Version: 1) => throttle_time_ms [results] 
  throttle_time_ms => INT32
  results => error_code log_dir [topics] 
    error_code => INT16
    log_dir => STRING
    topics => name [partitions] 
      name => STRING
      partitions => partition_index partition_size offset_lag is_future_key 
        partition_index => INT32
        partition_size => INT64
        offset_lag => INT64
        is_future_key => BOOLEAN
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
resultsThe log directories.
error_codeエラーコード。あるいはエラーが無かった場合は0。
log_dirログディレクトリの絶対パス。
topics各トピック。
名前トピック名。
partitions
partition_indexパーティションのインデックス。
partition_sizeこのパーティション内のログセグメントのサイズのバイト。
offset_lagログのLEOに関連する遅延。(パーティションについて現在のログであれば)パーティションのHW、あるいは(パーティションについての未来のログであれば)現在のレプリカのLEO
is_future_keyこのログが AlterReplicaLogDirsRequest によって生成され、将来レプリカの現在のログに置き換えられる場合true。

DescribeLogDirs Response (Version: 2) => throttle_time_ms [results] TAG_BUFFER 
  throttle_time_ms => INT32
  results => error_code log_dir [topics] TAG_BUFFER 
    error_code => INT16
    log_dir => COMPACT_STRING
    topics => name [partitions] TAG_BUFFER 
      name => COMPACT_STRING
      partitions => partition_index partition_size offset_lag is_future_key TAG_BUFFER 
        partition_index => INT32
        partition_size => INT64
        offset_lag => INT64
        is_future_key => BOOLEAN
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
resultsThe log directories.
error_codeエラーコード。あるいはエラーが無かった場合は0。
log_dirログディレクトリの絶対パス。
topics各トピック。
名前トピック名。
partitions
partition_indexパーティションのインデックス。
partition_sizeこのパーティション内のログセグメントのサイズのバイト。
offset_lagログのLEOに関連する遅延。(パーティションについて現在のログであれば)パーティションのHW、あるいは(パーティションについての未来のログであれば)現在のレプリカのLEO
is_future_keyこのログが AlterReplicaLogDirsRequest によって生成され、将来レプリカの現在のログに置き換えられる場合true。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

SaslAuthenticate API (Key: 36):
Requests:

SaslAuthenticate Request (Version: 0) => auth_bytes 
  auth_bytes => BYTES
フィールド 説明
auth_bytesSASL機構によって定義されたクライアントからのSASL認証バイト。

SaslAuthenticate Request (Version: 1) => auth_bytes 
  auth_bytes => BYTES
フィールド 説明
auth_bytesSASL機構によって定義されたクライアントからのSASL認証バイト。

SaslAuthenticate Request (Version: 2) => auth_bytes TAG_BUFFER 
  auth_bytes => COMPACT_BYTES
フィールド 説明
auth_bytesSASL機構によって定義されたクライアントからのSASL認証バイト。
_tagged_fieldsタグ付きのフィールド

Responses:

SaslAuthenticate Response (Version: 0) => error_code error_message auth_bytes 
  error_code => INT16
  error_message => NULLABLE_STRING
  auth_bytes => BYTES
フィールド 説明
error_codeエラーコード。あるいはエラーが無かった場合は0。
error_messageエラーメッセージ。あるいはエラーが無かった場合はnull。
auth_bytesSASL機構によって定義されたサーバからのSASL認証バイト。

SaslAuthenticate Response (Version: 1) => error_code error_message auth_bytes session_lifetime_ms 
  error_code => INT16
  error_message => NULLABLE_STRING
  auth_bytes => BYTES
  session_lifetime_ms => INT64
フィールド 説明
error_codeエラーコード。あるいはエラーが無かった場合は0。
error_messageエラーメッセージ。あるいはエラーが無かった場合はnull。
auth_bytesSASL機構によって定義されたサーバからのSASL認証バイト。
session_lifetime_msSASL機構によって定義されたサーバからのSASL認証バイト。

SaslAuthenticate Response (Version: 2) => error_code error_message auth_bytes session_lifetime_ms TAG_BUFFER 
  error_code => INT16
  error_message => COMPACT_NULLABLE_STRING
  auth_bytes => COMPACT_BYTES
  session_lifetime_ms => INT64
フィールド 説明
error_codeエラーコード。あるいはエラーが無かった場合は0。
error_messageエラーメッセージ。あるいはエラーが無かった場合はnull。
auth_bytesSASL機構によって定義されたサーバからのSASL認証バイト。
session_lifetime_msSASL機構によって定義されたサーバからのSASL認証バイト。
_tagged_fieldsタグ付きのフィールド

CreatePartitions API (Key: 37):
Requests:

CreatePartitions Request (Version: 0) => [topics] timeout_ms validate_only 
  topics => name count [assignments] 
    name => STRING
    count => INT32
    assignments => [broker_ids] 
      broker_ids => INT32
  timeout_ms => INT32
  validate_only => BOOLEAN
フィールド 説明
topics中に新しいパーティションを作成しようとする各トピック。
名前トピック名。
count新しいパーティションのカウント。
assignments新しいパーティションの割り当て。
broker_ids割り当てられたブローカー ID。
timeout_msパーティションが作成されるまで待つ時間のミリ秒。
validate_onlytrue の場合、リクエストを検証しますが、実際にはパーティションの番号を増やしません。

CreatePartitions Request (Version: 1) => [topics] timeout_ms validate_only 
  topics => name count [assignments] 
    name => STRING
    count => INT32
    assignments => [broker_ids] 
      broker_ids => INT32
  timeout_ms => INT32
  validate_only => BOOLEAN
フィールド 説明
topics中に新しいパーティションを作成しようとする各トピック。
名前トピック名。
count新しいパーティションのカウント。
assignments新しいパーティションの割り当て。
broker_ids割り当てられたブローカー ID。
timeout_msパーティションが作成されるまで待つ時間のミリ秒。
validate_onlytrue の場合、リクエストを検証しますが、実際にはパーティションの番号を増やしません。

CreatePartitions Request (Version: 2) => [topics] timeout_ms validate_only TAG_BUFFER 
  topics => name count [assignments] TAG_BUFFER 
    name => COMPACT_STRING
    count => INT32
    assignments => [broker_ids] TAG_BUFFER 
      broker_ids => INT32
  timeout_ms => INT32
  validate_only => BOOLEAN
フィールド 説明
topics中に新しいパーティションを作成しようとする各トピック。
名前トピック名。
count新しいパーティションのカウント。
assignments新しいパーティションの割り当て。
broker_ids割り当てられたブローカー ID。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド
timeout_msパーティションが作成されるまで待つ時間のミリ秒。
validate_onlytrue の場合、リクエストを検証しますが、実際にはパーティションの番号を増やしません。
_tagged_fieldsタグ付きのフィールド

CreatePartitions Request (Version: 3) => [topics] timeout_ms validate_only TAG_BUFFER 
  topics => name count [assignments] TAG_BUFFER 
    name => COMPACT_STRING
    count => INT32
    assignments => [broker_ids] TAG_BUFFER 
      broker_ids => INT32
  timeout_ms => INT32
  validate_only => BOOLEAN
フィールド 説明
topics中に新しいパーティションを作成しようとする各トピック。
名前トピック名。
count新しいパーティションのカウント。
assignments新しいパーティションの割り当て。
broker_ids割り当てられたブローカー ID。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド
timeout_msパーティションが作成されるまで待つ時間のミリ秒。
validate_onlytrue の場合、リクエストを検証しますが、実際にはパーティションの番号を増やしません。
_tagged_fieldsタグ付きのフィールド

Responses:

CreatePartitions Response (Version: 0) => throttle_time_ms [results] 
  throttle_time_ms => INT32
  results => name error_code error_message 
    name => STRING
    error_code => INT16
    error_message => NULLABLE_STRING
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
results各トピックのパーティション作成結果。
名前トピック名。
error_code結果エラー。あるいはエラーが無かった場合は0。
error_messageエラーメッセージ。あるいはエラーが無かった場合はnull。

CreatePartitions Response (Version: 1) => throttle_time_ms [results] 
  throttle_time_ms => INT32
  results => name error_code error_message 
    name => STRING
    error_code => INT16
    error_message => NULLABLE_STRING
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
results各トピックのパーティション作成結果。
名前トピック名。
error_code結果エラー。あるいはエラーが無かった場合は0。
error_messageエラーメッセージ。あるいはエラーが無かった場合はnull。

CreatePartitions Response (Version: 2) => throttle_time_ms [results] TAG_BUFFER 
  throttle_time_ms => INT32
  results => name error_code error_message TAG_BUFFER 
    name => COMPACT_STRING
    error_code => INT16
    error_message => COMPACT_NULLABLE_STRING
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
results各トピックのパーティション作成結果。
名前トピック名。
error_code結果エラー。あるいはエラーが無かった場合は0。
error_messageエラーメッセージ。あるいはエラーが無かった場合はnull。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

CreatePartitions Response (Version: 3) => throttle_time_ms [results] TAG_BUFFER 
  throttle_time_ms => INT32
  results => name error_code error_message TAG_BUFFER 
    name => COMPACT_STRING
    error_code => INT16
    error_message => COMPACT_NULLABLE_STRING
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
results各トピックのパーティション作成結果。
名前トピック名。
error_code結果エラー。あるいはエラーが無かった場合は0。
error_messageエラーメッセージ。あるいはエラーが無かった場合はnull。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

CreateDelegationToken API (Key: 38):
Requests:

CreateDelegationToken Request (Version: 0) => [renewers] max_lifetime_ms 
  renewers => principal_type principal_name 
    principal_type => STRING
    principal_name => STRING
  max_lifetime_ms => INT64
フィールド 説明
renewers有効期限が切れる前にこのトークンを更新できるユーザのリスト。
principal_typeKafka プリンシパルの型。
principal_nameKafka プリンシパルの名前。
max_lifetime_msトークンの最大持続時間(ミリ秒)。あるいは、サーバ側のデフォルトを使う場合は -1。

CreateDelegationToken Request (Version: 1) => [renewers] max_lifetime_ms 
  renewers => principal_type principal_name 
    principal_type => STRING
    principal_name => STRING
  max_lifetime_ms => INT64
フィールド 説明
renewers有効期限が切れる前にこのトークンを更新できるユーザのリスト。
principal_typeKafka プリンシパルの型。
principal_nameKafka プリンシパルの名前。
max_lifetime_msトークンの最大持続時間(ミリ秒)。あるいは、サーバ側のデフォルトを使う場合は -1。

CreateDelegationToken Request (Version: 2) => [renewers] max_lifetime_ms TAG_BUFFER 
  renewers => principal_type principal_name TAG_BUFFER 
    principal_type => COMPACT_STRING
    principal_name => COMPACT_STRING
  max_lifetime_ms => INT64
フィールド 説明
renewers有効期限が切れる前にこのトークンを更新できるユーザのリスト。
principal_typeKafka プリンシパルの型。
principal_nameKafka プリンシパルの名前。
_tagged_fieldsタグ付きのフィールド
max_lifetime_msトークンの最大持続時間(ミリ秒)。あるいは、サーバ側のデフォルトを使う場合は -1。
_tagged_fieldsタグ付きのフィールド

Responses:

CreateDelegationToken Response (Version: 0) => error_code principal_type principal_name issue_timestamp_ms expiry_timestamp_ms max_timestamp_ms token_id hmac throttle_time_ms 
  error_code => INT16
  principal_type => STRING
  principal_name => STRING
  issue_timestamp_ms => INT64
  expiry_timestamp_ms => INT64
  max_timestamp_ms => INT64
  token_id => STRING
  hmac => BYTES
  throttle_time_ms => INT32
フィールド 説明
error_codeトップレベルのエラー。あるいはエラーが無い場合は 0。
principal_typeトークンの所有者のプリンシパル型。
principal_nameトークンの所有者の名前。
issue_timestamp_msこのトークンが生成された時間。
expiry_timestamp_msこのトークンが有効期限切れになる時間。
max_timestamp_msこのトークンの最大持続時間。
token_idトークンの UUID。
hmac移譲トークンのHMAC。
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。

CreateDelegationToken Response (Version: 1) => error_code principal_type principal_name issue_timestamp_ms expiry_timestamp_ms max_timestamp_ms token_id hmac throttle_time_ms 
  error_code => INT16
  principal_type => STRING
  principal_name => STRING
  issue_timestamp_ms => INT64
  expiry_timestamp_ms => INT64
  max_timestamp_ms => INT64
  token_id => STRING
  hmac => BYTES
  throttle_time_ms => INT32
フィールド 説明
error_codeトップレベルのエラー。あるいはエラーが無い場合は 0。
principal_typeトークンの所有者のプリンシパル型。
principal_nameトークンの所有者の名前。
issue_timestamp_msこのトークンが生成された時間。
expiry_timestamp_msこのトークンが有効期限切れになる時間。
max_timestamp_msこのトークンの最大持続時間。
token_idトークンの UUID。
hmac移譲トークンのHMAC。
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。

CreateDelegationToken Response (Version: 2) => error_code principal_type principal_name issue_timestamp_ms expiry_timestamp_ms max_timestamp_ms token_id hmac throttle_time_ms TAG_BUFFER 
  error_code => INT16
  principal_type => COMPACT_STRING
  principal_name => COMPACT_STRING
  issue_timestamp_ms => INT64
  expiry_timestamp_ms => INT64
  max_timestamp_ms => INT64
  token_id => COMPACT_STRING
  hmac => COMPACT_BYTES
  throttle_time_ms => INT32
フィールド 説明
error_codeトップレベルのエラー。あるいはエラーが無い場合は 0。
principal_typeトークンの所有者のプリンシパル型。
principal_nameトークンの所有者の名前。
issue_timestamp_msこのトークンが生成された時間。
expiry_timestamp_msこのトークンが有効期限切れになる時間。
max_timestamp_msこのトークンの最大持続時間。
token_idトークンの UUID。
hmac移譲トークンのHMAC。
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
_tagged_fieldsタグ付きのフィールド

RenewDelegationToken API (Key: 39):
Requests:

RenewDelegationToken Request (Version: 0) => hmac renew_period_ms 
  hmac => BYTES
  renew_period_ms => INT64
フィールド 説明
hmac更新する移譲トークンの HMAC。
renew_period_ms更新時間のミリ秒。

RenewDelegationToken Request (Version: 1) => hmac renew_period_ms 
  hmac => BYTES
  renew_period_ms => INT64
フィールド 説明
hmac更新する移譲トークンの HMAC。
renew_period_ms更新時間のミリ秒。

RenewDelegationToken Request (Version: 2) => hmac renew_period_ms TAG_BUFFER 
  hmac => COMPACT_BYTES
  renew_period_ms => INT64
フィールド 説明
hmac更新する移譲トークンの HMAC。
renew_period_ms更新時間のミリ秒。
_tagged_fieldsタグ付きのフィールド

Responses:

RenewDelegationToken Response (Version: 0) => error_code expiry_timestamp_ms throttle_time_ms 
  error_code => INT16
  expiry_timestamp_ms => INT64
  throttle_time_ms => INT32
フィールド 説明
error_codeエラーコード。あるいはエラーが無かった場合は0。
expiry_timestamp_msこのトークンが有効期限になる時のタイムスタンプ(ミリ秒)。
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。

RenewDelegationToken Response (Version: 1) => error_code expiry_timestamp_ms throttle_time_ms 
  error_code => INT16
  expiry_timestamp_ms => INT64
  throttle_time_ms => INT32
フィールド 説明
error_codeエラーコード。あるいはエラーが無かった場合は0。
expiry_timestamp_msこのトークンが有効期限になる時のタイムスタンプ(ミリ秒)。
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。

RenewDelegationToken Response (Version: 2) => error_code expiry_timestamp_ms throttle_time_ms TAG_BUFFER 
  error_code => INT16
  expiry_timestamp_ms => INT64
  throttle_time_ms => INT32
フィールド 説明
error_codeエラーコード。あるいはエラーが無かった場合は0。
expiry_timestamp_msこのトークンが有効期限になる時のタイムスタンプ(ミリ秒)。
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
_tagged_fieldsタグ付きのフィールド

ExpireDelegationToken API (Key: 40):
Requests:

ExpireDelegationToken Request (Version: 0) => hmac expiry_time_period_ms 
  hmac => BYTES
  expiry_time_period_ms => INT64
フィールド 説明
hmac期限切れにする移譲トークンのHMAC。
expiry_time_period_ms有効期限のミリ秒。

ExpireDelegationToken Request (Version: 1) => hmac expiry_time_period_ms 
  hmac => BYTES
  expiry_time_period_ms => INT64
フィールド 説明
hmac期限切れにする移譲トークンのHMAC。
expiry_time_period_ms有効期限のミリ秒。

ExpireDelegationToken Request (Version: 2) => hmac expiry_time_period_ms TAG_BUFFER 
  hmac => COMPACT_BYTES
  expiry_time_period_ms => INT64
フィールド 説明
hmac期限切れにする移譲トークンのHMAC。
expiry_time_period_ms有効期限のミリ秒。
_tagged_fieldsタグ付きのフィールド

Responses:

ExpireDelegationToken Response (Version: 0) => error_code expiry_timestamp_ms throttle_time_ms 
  error_code => INT16
  expiry_timestamp_ms => INT64
  throttle_time_ms => INT32
フィールド 説明
error_codeエラーコード。あるいはエラーが無かった場合は0。
expiry_timestamp_msこのトークンが有効期限になる時のタイムスタンプ(ミリ秒)。
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。

ExpireDelegationToken Response (Version: 1) => error_code expiry_timestamp_ms throttle_time_ms 
  error_code => INT16
  expiry_timestamp_ms => INT64
  throttle_time_ms => INT32
フィールド 説明
error_codeエラーコード。あるいはエラーが無かった場合は0。
expiry_timestamp_msこのトークンが有効期限になる時のタイムスタンプ(ミリ秒)。
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。

ExpireDelegationToken Response (Version: 2) => error_code expiry_timestamp_ms throttle_time_ms TAG_BUFFER 
  error_code => INT16
  expiry_timestamp_ms => INT64
  throttle_time_ms => INT32
フィールド 説明
error_codeエラーコード。あるいはエラーが無かった場合は0。
expiry_timestamp_msこのトークンが有効期限になる時のタイムスタンプ(ミリ秒)。
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
_tagged_fieldsタグ付きのフィールド

DescribeDelegationToken API (Key: 41):
Requests:

DescribeDelegationToken Request (Version: 0) => [owners] 
  owners => principal_type principal_name 
    principal_type => STRING
    principal_name => STRING
フィールド 説明
owners移譲トークンを説明する各所有者。あるいは全てのトークンを説明するには null。
principal_type所有者のプリンシパル型。
principal_name所有者のプリンシパル名。

DescribeDelegationToken Request (Version: 1) => [owners] 
  owners => principal_type principal_name 
    principal_type => STRING
    principal_name => STRING
フィールド 説明
owners移譲トークンを説明する各所有者。あるいは全てのトークンを説明するには null。
principal_type所有者のプリンシパル型。
principal_name所有者のプリンシパル名。

DescribeDelegationToken Request (Version: 2) => [owners] TAG_BUFFER 
  owners => principal_type principal_name TAG_BUFFER 
    principal_type => COMPACT_STRING
    principal_name => COMPACT_STRING
フィールド 説明
owners移譲トークンを説明する各所有者。あるいは全てのトークンを説明するには null。
principal_type所有者のプリンシパル型。
principal_name所有者のプリンシパル名。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

Responses:

DescribeDelegationToken Response (Version: 0) => error_code [tokens] throttle_time_ms 
  error_code => INT16
  tokens => principal_type principal_name issue_timestamp expiry_timestamp max_timestamp token_id hmac [renewers] 
    principal_type => STRING
    principal_name => STRING
    issue_timestamp => INT64
    expiry_timestamp => INT64
    max_timestamp => INT64
    token_id => STRING
    hmac => BYTES
    renewers => principal_type principal_name 
      principal_type => STRING
      principal_name => STRING
  throttle_time_ms => INT32
フィールド 説明
error_codeエラーコード。あるいはエラーが無かった場合は0。
tokensトークン。
principal_typeトークンのプリンシパル型。
principal_nameトークンのプリンシパル名。
issue_timestampトークンの発行タイムスタンプのミリ秒。
expiry_timestampトークンの有効期限のタイムスタンプのミリ秒。
max_timestampトークンの最大タイムスタンプ長のミリ秒。
token_idトークン ID。
hmacトークンの HMAC。
renewersこのトークンが有効期限切れになる前にこのトークンを更新できるユーザ。
principal_typeThe renewer principal type
principal_nameThe renewer principal name
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。

DescribeDelegationToken Response (Version: 1) => error_code [tokens] throttle_time_ms 
  error_code => INT16
  tokens => principal_type principal_name issue_timestamp expiry_timestamp max_timestamp token_id hmac [renewers] 
    principal_type => STRING
    principal_name => STRING
    issue_timestamp => INT64
    expiry_timestamp => INT64
    max_timestamp => INT64
    token_id => STRING
    hmac => BYTES
    renewers => principal_type principal_name 
      principal_type => STRING
      principal_name => STRING
  throttle_time_ms => INT32
フィールド 説明
error_codeエラーコード。あるいはエラーが無かった場合は0。
tokensトークン。
principal_typeトークンのプリンシパル型。
principal_nameトークンのプリンシパル名。
issue_timestampトークンの発行タイムスタンプのミリ秒。
expiry_timestampトークンの有効期限のタイムスタンプのミリ秒。
max_timestampトークンの最大タイムスタンプ長のミリ秒。
token_idトークン ID。
hmacトークンの HMAC。
renewersこのトークンが有効期限切れになる前にこのトークンを更新できるユーザ。
principal_typeThe renewer principal type
principal_nameThe renewer principal name
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。

DescribeDelegationToken Response (Version: 2) => error_code [tokens] throttle_time_ms TAG_BUFFER 
  error_code => INT16
  tokens => principal_type principal_name issue_timestamp expiry_timestamp max_timestamp token_id hmac [renewers] TAG_BUFFER 
    principal_type => COMPACT_STRING
    principal_name => COMPACT_STRING
    issue_timestamp => INT64
    expiry_timestamp => INT64
    max_timestamp => INT64
    token_id => COMPACT_STRING
    hmac => COMPACT_BYTES
    renewers => principal_type principal_name TAG_BUFFER 
      principal_type => COMPACT_STRING
      principal_name => COMPACT_STRING
  throttle_time_ms => INT32
フィールド 説明
error_codeエラーコード。あるいはエラーが無かった場合は0。
tokensトークン。
principal_typeトークンのプリンシパル型。
principal_nameトークンのプリンシパル名。
issue_timestampトークンの発行タイムスタンプのミリ秒。
expiry_timestampトークンの有効期限のタイムスタンプのミリ秒。
max_timestampトークンの最大タイムスタンプ長のミリ秒。
token_idトークン ID。
hmacトークンの HMAC。
renewersこのトークンが有効期限切れになる前にこのトークンを更新できるユーザ。
principal_typeThe renewer principal type
principal_nameThe renewer principal name
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
_tagged_fieldsタグ付きのフィールド

DeleteGroups API (Key: 42):
Requests:

DeleteGroups Request (Version: 0) => [groups_names] 
  groups_names => STRING
フィールド 説明
groups_names削除するグループ名。

DeleteGroups Request (Version: 1) => [groups_names] 
  groups_names => STRING
フィールド 説明
groups_names削除するグループ名。

DeleteGroups Request (Version: 2) => [groups_names] TAG_BUFFER 
  groups_names => COMPACT_STRING
フィールド 説明
groups_names削除するグループ名。
_tagged_fieldsタグ付きのフィールド

Responses:

DeleteGroups Response (Version: 0) => throttle_time_ms [results] 
  throttle_time_ms => INT32
  results => group_id error_code 
    group_id => STRING
    error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
results削除結果。
group_idグループ id
error_code削除エラー。あるいは削除が成功した場合は0。

DeleteGroups Response (Version: 1) => throttle_time_ms [results] 
  throttle_time_ms => INT32
  results => group_id error_code 
    group_id => STRING
    error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
results削除結果。
group_idグループ id
error_code削除エラー。あるいは削除が成功した場合は0。

DeleteGroups Response (Version: 2) => throttle_time_ms [results] TAG_BUFFER 
  throttle_time_ms => INT32
  results => group_id error_code TAG_BUFFER 
    group_id => COMPACT_STRING
    error_code => INT16
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
results削除結果。
group_idグループ id
error_code削除エラー。あるいは削除が成功した場合は0。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

ElectLeaders API (Key: 43):
Requests:

ElectLeaders Request (Version: 0) => [topic_partitions] timeout_ms 
  topic_partitions => topic [partition_id] 
    topic => STRING
    partition_id => INT32
  timeout_ms => INT32
フィールド 説明
topic_partitionsリーダー選出するトピックパーティション。
トピックトピック名。
partition_idリーダーが選出されるこのトピックのパーティション。
timeout_ms選出が完了する前で待つ時間のミリ秒。

ElectLeaders Request (Version: 1) => election_type [topic_partitions] timeout_ms 
  election_type => INT8
  topic_partitions => topic [partition_id] 
    topic => STRING
    partition_id => INT32
  timeout_ms => INT32
フィールド 説明
election_typeType of elections to conduct for the partition. A value of '0' elects the preferred replica. A value of '1' elects the first live replica if there are no in-sync replica.
topic_partitionsリーダー選出するトピックパーティション。
トピックトピック名。
partition_idリーダーが選出されるこのトピックのパーティション。
timeout_ms選出が完了する前で待つ時間のミリ秒。

ElectLeaders Request (Version: 2) => election_type [topic_partitions] timeout_ms TAG_BUFFER 
  election_type => INT8
  topic_partitions => topic [partition_id] TAG_BUFFER 
    topic => COMPACT_STRING
    partition_id => INT32
  timeout_ms => INT32
フィールド 説明
election_typeType of elections to conduct for the partition. A value of '0' elects the preferred replica. A value of '1' elects the first live replica if there are no in-sync replica.
topic_partitionsリーダー選出するトピックパーティション。
トピックトピック名。
partition_idリーダーが選出されるこのトピックのパーティション。
_tagged_fieldsタグ付きのフィールド
timeout_ms選出が完了する前で待つ時間のミリ秒。
_tagged_fieldsタグ付きのフィールド

Responses:

ElectLeaders Response (Version: 0) => throttle_time_ms [replica_election_results] 
  throttle_time_ms => INT32
  replica_election_results => topic [partition_result] 
    topic => STRING
    partition_result => partition_id error_code error_message 
      partition_id => INT32
      error_code => INT16
      error_message => NULLABLE_STRING
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
replica_election_results選出結果。あるいはリクエストしたものが権限を持っておらず、リクエストが全てのパーティションに尋ねる場合は空の配列。
トピックトピック名。
partition_result各パーティションの結果
partition_idパーティション id
error_code結果エラー。あるいはエラーが無かった場合は0。
error_messageエラーメッセージ。あるいはエラーが無かった場合はnull。

ElectLeaders Response (Version: 1) => throttle_time_ms error_code [replica_election_results] 
  throttle_time_ms => INT32
  error_code => INT16
  replica_election_results => topic [partition_result] 
    topic => STRING
    partition_result => partition_id error_code error_message 
      partition_id => INT32
      error_code => INT16
      error_message => NULLABLE_STRING
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeトップレベルの応答エラーコード。
replica_election_results選出結果。あるいはリクエストしたものが権限を持っておらず、リクエストが全てのパーティションに尋ねる場合は空の配列。
トピックトピック名。
partition_result各パーティションの結果
partition_idパーティション id
error_code結果エラー。あるいはエラーが無かった場合は0。
error_messageエラーメッセージ。あるいはエラーが無かった場合はnull。

ElectLeaders Response (Version: 2) => throttle_time_ms error_code [replica_election_results] TAG_BUFFER 
  throttle_time_ms => INT32
  error_code => INT16
  replica_election_results => topic [partition_result] TAG_BUFFER 
    topic => COMPACT_STRING
    partition_result => partition_id error_code error_message TAG_BUFFER 
      partition_id => INT32
      error_code => INT16
      error_message => COMPACT_NULLABLE_STRING
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeトップレベルの応答エラーコード。
replica_election_results選出結果。あるいはリクエストしたものが権限を持っておらず、リクエストが全てのパーティションに尋ねる場合は空の配列。
トピックトピック名。
partition_result各パーティションの結果
partition_idパーティション id
error_code結果エラー。あるいはエラーが無かった場合は0。
error_messageエラーメッセージ。あるいはエラーが無かった場合はnull。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

IncrementalAlterConfigs API (Key: 44):
Requests:

IncrementalAlterConfigs Request (Version: 0) => [resources] validate_only 
  resources => resource_type resource_name [configs] 
    resource_type => INT8
    resource_name => STRING
    configs => name config_operation value 
      name => STRING
      config_operation => INT8
      value => NULLABLE_STRING
  validate_only => BOOLEAN
フィールド 説明
resources各リソースについての逐次的な更新。
resource_typeリソース型。
resource_nameリソース名。
configs設定。
名前設定キー名。
config_operation操作の型 (Set, Delete, Append, Subtract)。
設定キーに設定する値。
validate_onlyリクエストを検証する必要があるが、設定を変更しない場合は true。

IncrementalAlterConfigs Request (Version: 1) => [resources] validate_only TAG_BUFFER 
  resources => resource_type resource_name [configs] TAG_BUFFER 
    resource_type => INT8
    resource_name => COMPACT_STRING
    configs => name config_operation value TAG_BUFFER 
      name => COMPACT_STRING
      config_operation => INT8
      value => COMPACT_NULLABLE_STRING
  validate_only => BOOLEAN
フィールド 説明
resources各リソースについての逐次的な更新。
resource_typeリソース型。
resource_nameリソース名。
configs設定。
名前設定キー名。
config_operation操作の型 (Set, Delete, Append, Subtract)。
設定キーに設定する値。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド
validate_onlyリクエストを検証する必要があるが、設定を変更しない場合は true。
_tagged_fieldsタグ付きのフィールド

Responses:

IncrementalAlterConfigs Response (Version: 0) => throttle_time_ms [responses] 
  throttle_time_ms => INT32
  responses => error_code error_message resource_type resource_name 
    error_code => INT16
    error_message => NULLABLE_STRING
    resource_type => INT8
    resource_name => STRING
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
responses各リソースについての応答。
error_codeリソース エラーコード。
error_messageリソース エラー メッセージ。あるいはエラーが無かった場合はnull。
resource_typeリソース型。
resource_nameリソース名。

IncrementalAlterConfigs Response (Version: 1) => throttle_time_ms [responses] TAG_BUFFER 
  throttle_time_ms => INT32
  responses => error_code error_message resource_type resource_name TAG_BUFFER 
    error_code => INT16
    error_message => COMPACT_NULLABLE_STRING
    resource_type => INT8
    resource_name => COMPACT_STRING
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
responses各リソースについての応答。
error_codeリソース エラーコード。
error_messageリソース エラー メッセージ。あるいはエラーが無かった場合はnull。
resource_typeリソース型。
resource_nameリソース名。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

AlterPartitionReassignments API (Key: 45):
Requests:

AlterPartitionReassignments Request (Version: 0) => timeout_ms [topics] TAG_BUFFER 
  timeout_ms => INT32
  topics => name [partitions] TAG_BUFFER 
    name => COMPACT_STRING
    partitions => partition_index [replicas] TAG_BUFFER 
      partition_index => INT32
      replicas => INT32
フィールド 説明
timeout_msリクエストが完了するまで待つ時間のミリ秒。
topics再割り当てするトピック。
名前トピック名。
partitions再割り当てするパーティション。
partition_indexパーティションのインデックス。
replicasパーティションを配置するレプリカ。あるいは、このパーティションで保留中の再割り当てを中止するには null。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

Responses:

AlterPartitionReassignments Response (Version: 0) => throttle_time_ms error_code error_message [responses] TAG_BUFFER 
  throttle_time_ms => INT32
  error_code => INT16
  error_message => COMPACT_NULLABLE_STRING
  responses => name [partitions] TAG_BUFFER 
    name => COMPACT_STRING
    partitions => partition_index error_code error_message TAG_BUFFER 
      partition_index => INT32
      error_code => INT16
      error_message => COMPACT_NULLABLE_STRING
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeトップレベルのエラーコード。あるいはエラーが無い場合は 0。
error_messageトップレベルのエラーメッセージ。あるいは、エラーが無かった場合は null。
responses再割り当てするトピックへの応答。
名前トピック名。
partitions再割り当てするパーティションへの応答。
partition_indexパーティションのインデックス。
error_codeこのパーティションのエラーコード。あるいはエラーが無い場合は 0。
error_messageこのパーティションのエラーメッセージ。あるいは、エラーが無かった場合は null。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

ListPartitionReassignments API (Key: 46):
Requests:

ListPartitionReassignments Request (Version: 0) => timeout_ms [topics] TAG_BUFFER 
  timeout_ms => INT32
  topics => name [partition_indexes] TAG_BUFFER 
    name => COMPACT_STRING
    partition_indexes => INT32
フィールド 説明
timeout_msリクエストが完了するまで待つ時間のミリ秒。
topicsパーティションの再割り当てをリストするトピック。あるいは、全てをリストするには null。
名前トピック名。
partition_indexesパーティションの再割り当てをリストするパーティション。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

Responses:

ListPartitionReassignments Response (Version: 0) => throttle_time_ms error_code error_message [topics] TAG_BUFFER 
  throttle_time_ms => INT32
  error_code => INT16
  error_message => COMPACT_NULLABLE_STRING
  topics => name [partitions] TAG_BUFFER 
    name => COMPACT_STRING
    partitions => partition_index [replicas] [adding_replicas] [removing_replicas] TAG_BUFFER 
      partition_index => INT32
      replicas => INT32
      adding_replicas => INT32
      removing_replicas => INT32
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeトップレベルのエラーコード。あるいはエラーが無い場合は 0。
error_messageトップレベルのエラーメッセージ。あるいは、エラーが無かった場合は null。
topics各トピックで実行中の再割り当て。
名前トピック名。
partitions各パーティションで実行中の再割り当て。
partition_indexパーティションのインデックス。
replicas現在のレプリカセット。
adding_replicas現在追加しているレプリカのセット。
removing_replicas現在削除しているレプリカのセット。
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

OffsetDelete API (Key: 47):
Requests:

OffsetDelete Request (Version: 0) => group_id [topics] 
  group_id => STRING
  topics => name [partitions] 
    name => STRING
    partitions => partition_index 
      partition_index => INT32
フィールド 説明
group_idユニークなグループの識別子。
topicsオフセットを削除するトピック
名前トピック名。
partitionsオフセットを削除する各パーティション。
partition_indexパーティションのインデックス。

Responses:

OffsetDelete Response (Version: 0) => error_code throttle_time_ms [topics] 
  error_code => INT16
  throttle_time_ms => INT32
  topics => name [partitions] 
    name => STRING
    partitions => partition_index error_code 
      partition_index => INT32
      error_code => INT16
フィールド 説明
error_codeトップレベルのエラーコード。あるいはエラーが無い場合は 0。
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
topics各トピックについての応答。
名前トピック名。
partitionsトピック内の各パーティションについての応答。
partition_indexパーティションのインデックス。
error_codeエラーコード。あるいはエラーが無かった場合は0。

DescribeClientQuotas API (Key: 48):
Requests:

DescribeClientQuotas Request (Version: 0) => [components] strict 
  components => entity_type match_type match 
    entity_type => STRING
    match_type => INT8
    match => NULLABLE_STRING
  strict => BOOLEAN
フィールド 説明
componentsコンポーネントをフィルタして、クォータエンティティに適用します。
entity_typeフィルターコンポ―ネントが適用されるエンティティ型。
match_typeHow to match the entity {0 = exact name, 1 = default name, 2 = any specified name}.
matchThe string to match against, or null if unused for the match type.
strictWhether the match is strict, i.e. should exclude entities with unspecified entity types.

Responses:

DescribeClientQuotas Response (Version: 0) => throttle_time_ms error_code error_message [entries] 
  throttle_time_ms => INT32
  error_code => INT16
  error_message => NULLABLE_STRING
  entries => [entity] [values] 
    entity => entity_type entity_name 
      entity_type => STRING
      entity_name => NULLABLE_STRING
    values => key value 
      key => STRING
      value => FLOAT64
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeThe error code, or `0` if the quota description succeeded.
error_messageThe error message, or `null` if the quota description succeeded.
entriesA result entry.
entityThe quota entity description.
entity_typeThe entity type.
entity_nameThe entity name, or null if the default.
valuesThe quota values for the entity.
キーThe quota configuration key.
The quota configuration value.

AlterClientQuotas API (Key: 49):
Requests:

AlterClientQuotas Request (Version: 0) => [entries] validate_only 
  entries => [entity] [ops] 
    entity => entity_type entity_name 
      entity_type => STRING
      entity_name => NULLABLE_STRING
    ops => key value remove 
      key => STRING
      value => FLOAT64
      remove => BOOLEAN
  validate_only => BOOLEAN
フィールド 説明
entriesThe quota configuration entries to alter.
entityThe quota entity to alter.
entity_typeThe entity type.
entity_nameThe name of the entity, or null if the default.
opsAn individual quota configuration entry to alter.
キーThe quota configuration key.
The value to set, otherwise ignored if the value is to be removed.
removeWhether the quota configuration value should be removed, otherwise set.
validate_onlyWhether the alteration should be validated, but not performed.

Responses:

AlterClientQuotas Response (Version: 0) => throttle_time_ms [entries] 
  throttle_time_ms => INT32
  entries => error_code error_message [entity] 
    error_code => INT16
    error_message => NULLABLE_STRING
    entity => entity_type entity_name 
      entity_type => STRING
      entity_name => NULLABLE_STRING
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
entriesThe quota configuration entries to alter.
error_codeThe error code, or `0` if the quota alteration succeeded.
error_messageThe error message, or `null` if the quota alteration succeeded.
entityThe quota entity to alter.
entity_typeThe entity type.
entity_nameThe name of the entity, or null if the default.

DescribeUserScramCredentials API (Key: 50):
Requests:

DescribeUserScramCredentials Request (Version: 0) => [users] TAG_BUFFER 
  users => name TAG_BUFFER 
    name => COMPACT_STRING
フィールド 説明
usersThe users to describe, or null/empty to describe all users.
名前The user name.
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

Responses:

DescribeUserScramCredentials Response (Version: 0) => throttle_time_ms error_code error_message [results] TAG_BUFFER 
  throttle_time_ms => INT32
  error_code => INT16
  error_message => COMPACT_NULLABLE_STRING
  results => user error_code error_message [credential_infos] TAG_BUFFER 
    user => COMPACT_STRING
    error_code => INT16
    error_message => COMPACT_NULLABLE_STRING
    credential_infos => mechanism iterations TAG_BUFFER 
      mechanism => INT8
      iterations => INT32
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeThe message-level error code, 0 except for user authorization or infrastructure issues.
error_messageThe message-level error message, if any.
resultsThe results for descriptions, one per user.
userThe user name.
error_codeThe user-level error code.
error_messageThe user-level error message, if any.
credential_infosThe mechanism and related information associated with the user's SCRAM credentials.
mechanismThe SCRAM mechanism.
iterationsThe number of iterations used in the SCRAM credential.
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

AlterUserScramCredentials API (Key: 51):
Requests:

AlterUserScramCredentials Request (Version: 0) => [deletions] [upsertions] TAG_BUFFER 
  deletions => name mechanism TAG_BUFFER 
    name => COMPACT_STRING
    mechanism => INT8
  upsertions => name mechanism iterations salt salted_password TAG_BUFFER 
    name => COMPACT_STRING
    mechanism => INT8
    iterations => INT32
    salt => COMPACT_BYTES
    salted_password => COMPACT_BYTES
フィールド 説明
deletionsThe SCRAM credentials to remove.
名前The user name.
mechanismThe SCRAM mechanism.
_tagged_fieldsタグ付きのフィールド
upsertionsThe SCRAM credentials to update/insert.
名前The user name.
mechanismThe SCRAM mechanism.
iterationsThe number of iterations.
saltA random salt generated by the client.
salted_passwordThe salted password.
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

Responses:

AlterUserScramCredentials Response (Version: 0) => throttle_time_ms [results] TAG_BUFFER 
  throttle_time_ms => INT32
  results => user error_code error_message TAG_BUFFER 
    user => COMPACT_STRING
    error_code => INT16
    error_message => COMPACT_NULLABLE_STRING
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
resultsThe results for deletions and alterations, one per affected user.
userThe user name.
error_codeThe error code.
error_messageThe error message, if any.
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

AlterIsr API (Key: 56):
Requests:

AlterIsr Request (Version: 0) => broker_id broker_epoch [topics] TAG_BUFFER 
  broker_id => INT32
  broker_epoch => INT64
  topics => name [partitions] TAG_BUFFER 
    name => COMPACT_STRING
    partitions => partition_index leader_epoch [new_isr] current_isr_version TAG_BUFFER 
      partition_index => INT32
      leader_epoch => INT32
      new_isr => INT32
      current_isr_version => INT32
フィールド 説明
broker_idThe ID of the requesting broker
broker_epochThe epoch of the requesting broker
topics
名前The name of the topic to alter ISRs for
partitions
partition_indexThe partition index
leader_epochThe leader epoch of this partition
new_isrThe ISR for this partition
current_isr_versionThe expected version of ISR which is being updated
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

Responses:

AlterIsr Response (Version: 0) => throttle_time_ms error_code [topics] TAG_BUFFER 
  throttle_time_ms => INT32
  error_code => INT16
  topics => name [partitions] TAG_BUFFER 
    name => COMPACT_STRING
    partitions => partition_index error_code leader_id leader_epoch [isr] current_isr_version TAG_BUFFER 
      partition_index => INT32
      error_code => INT16
      leader_id => INT32
      leader_epoch => INT32
      isr => INT32
      current_isr_version => INT32
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeThe top level response error code
topics
名前The name of the topic
partitions
partition_indexThe partition index
error_codeThe partition level error code
leader_idリーダーのブローカー ID。
leader_epochリーダーのepoch。
isr同期中のレプリカ ID。
current_isr_versionThe current ISR version.
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

UpdateFeatures API (Key: 57):
Requests:

UpdateFeatures Request (Version: 0) => timeout_ms [feature_updates] TAG_BUFFER 
  timeout_ms => INT32
  feature_updates => feature max_version_level allow_downgrade TAG_BUFFER 
    feature => COMPACT_STRING
    max_version_level => INT16
    allow_downgrade => BOOLEAN
フィールド 説明
timeout_msリクエストをタイムアウトする前にどれだけ長く待つかのミリ秒。
feature_updatesThe list of updates to finalized features.
機能The name of the finalized feature to be updated.
max_version_levelThe new maximum version level for the finalized feature. A value >= 1 is valid. A value < 1, is special, and can be used to request the deletion of the finalized feature.
allow_downgradeWhen set to true, the finalized feature version level is allowed to be downgraded/deleted. The downgrade request will fail if the new maximum version level is a value that's not lower than the existing maximum finalized version level.
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

Responses:

UpdateFeatures Response (Version: 0) => throttle_time_ms error_code error_message [results] TAG_BUFFER 
  throttle_time_ms => INT32
  error_code => INT16
  error_message => COMPACT_NULLABLE_STRING
  results => feature error_code error_message TAG_BUFFER 
    feature => COMPACT_STRING
    error_code => INT16
    error_message => COMPACT_NULLABLE_STRING
フィールド 説明
throttle_time_ms割り当て量の違反によって絞られたリクエストのミリ秒の維持期間。あるいは、リクエストが何も割り当て量に違反しなかった場合は0。
error_codeThe top-level error code, or `0` if there was no top-level error.
error_messageThe top-level error message, or `null` if there was no top-level error.
resultsResults for each feature update.
機能The name of the finalized feature.
error_codeThe feature update error code or `0` if the feature update succeeded.
error_messageThe feature update error, or `null` if the feature update succeeded.
_tagged_fieldsタグ付きのフィールド
_tagged_fieldsタグ付きのフィールド

いくつかの一般的な冷静な質問

人々はなぜHTTPを使わないのかを尋ねました。多くの理由があります。最大の理由はクライアントの実装者がより進んだTCP機能を利用することができるということです --多重のリクエストの機能、同時に多くの接続をpollする機能など。多くの言語でHTTPライブラリが驚くほどみすぼらしいことをも知りました。

Others have asked if maybe we shouldn't support many different protocols. Prior experience with this was that it makes it very hard to add and test new features if they have to be ported across many protocol implementations. Our feeling is that most users don't really see multiple protocols as a feature, they just want a good reliable client in the language of their choice.

Another question is why we don't adopt XMPP, STOMP, AMQP or an existing protocol. The answer to this varies by protocol, but in general the problem is that the protocol does determine large parts of the implementation and we couldn't do what we are doing if we didn't have control over the protocol. Our belief is that it is possible to do better than existing messaging systems have in providing a truly distributed messaging system, and to do this we need to build something that works differently.

A final question is why we don't use a system like Protocol Buffers or Thrift to define our request messages. These packages excel at helping you to managing lots and lots of serialized messages. However we have only a few messages. Support across languages is somewhat spotty (depending on the package). Finally the mapping between binary log format and wire protocol is something we manage somewhat carefully and this would not be possible with these systems. Finally we prefer the style of versioning APIs explicitly and checking this to inferring new values as nulls as it allows more nuanced control of compatibility.

TOP
inserted by FC2 system