You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by ch...@apache.org on 2021/05/03 04:42:32 UTC
[kafka] branch trunk updated: MINOR: Remove duplicate method in
test classes (#10535)
This is an automated email from the ASF dual-hosted git repository.
chia7712 pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git
The following commit(s) were added to refs/heads/trunk by this push:
new 3ddc377 MINOR: Remove duplicate method in test classes (#10535)
3ddc377 is described below
commit 3ddc377b056bde60f549ef383f9a8af1152ac3d6
Author: dengziming <sw...@163.com>
AuthorDate: Mon May 3 12:41:03 2021 +0800
MINOR: Remove duplicate method in test classes (#10535)
1. Remove duplicate serializing auto-generated data in RequestConvertToJsonTest, this is inspired by #9964
2. Remove RequestTestUtils.serializeRequestWithHeader since we added a AbstractRequest.serializeWithHeader in #10142
Reviewers: Chia-Ping Tsai <ch...@gmail.com>
---
.../kafka/common/requests/RequestTestUtils.java | 4 ----
.../kafka/server/IntegrationTestUtils.scala | 8 ++++----
.../unit/kafka/network/RequestChannelTest.scala | 5 ++---
.../kafka/network/RequestConvertToJsonTest.scala | 23 +++++++---------------
.../unit/kafka/network/SocketServerTest.scala | 14 ++++++-------
.../server/AbstractApiVersionsRequestTest.scala | 10 ++++++----
.../kafka/server/BaseClientQuotaManagerTest.scala | 15 ++++++--------
.../scala/unit/kafka/server/BaseRequestTest.scala | 13 ++++++------
.../unit/kafka/server/ControllerApisTest.scala | 3 +--
.../unit/kafka/server/ForwardingManagerTest.scala | 2 +-
.../scala/unit/kafka/server/KafkaApisTest.scala | 10 +++++-----
.../scala/unit/kafka/server/RequestQuotaTest.scala | 4 ++--
12 files changed, 47 insertions(+), 64 deletions(-)
diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestTestUtils.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestTestUtils.java
index 3334821..4ec1171 100644
--- a/clients/src/test/java/org/apache/kafka/common/requests/RequestTestUtils.java
+++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestTestUtils.java
@@ -48,10 +48,6 @@ public class RequestTestUtils {
return buffer;
}
- public static ByteBuffer serializeRequestWithHeader(RequestHeader header, AbstractRequest request) {
- return RequestUtils.serialize(header.data(), header.headerVersion(), request.data(), request.version());
- }
-
public static ByteBuffer serializeResponseWithHeader(AbstractResponse response, short version, int correlationId) {
return response.serializeWithHeader(new ResponseHeader(correlationId,
response.apiKey().responseHeaderVersion(version)), version);
diff --git a/core/src/test/scala/integration/kafka/server/IntegrationTestUtils.scala b/core/src/test/scala/integration/kafka/server/IntegrationTestUtils.scala
index c240cbf..9a21e7f 100644
--- a/core/src/test/scala/integration/kafka/server/IntegrationTestUtils.scala
+++ b/core/src/test/scala/integration/kafka/server/IntegrationTestUtils.scala
@@ -21,7 +21,7 @@ import kafka.network.SocketServer
import kafka.utils.{NotNothing, TestUtils}
import org.apache.kafka.common.network.{ListenerName, Mode}
import org.apache.kafka.common.protocol.ApiKeys
-import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, RequestHeader, RequestTestUtils, ResponseHeader}
+import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, RequestHeader, ResponseHeader}
import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.utils.Utils
@@ -34,15 +34,15 @@ import scala.reflect.ClassTag
object IntegrationTestUtils {
- private def sendRequest(socket: Socket, request: Array[Byte]): Unit = {
+ def sendRequest(socket: Socket, request: Array[Byte]): Unit = {
val outgoing = new DataOutputStream(socket.getOutputStream)
outgoing.writeInt(request.length)
outgoing.write(request)
outgoing.flush()
}
- def sendWithHeader(request: AbstractRequest, header: RequestHeader, socket: Socket): Unit = {
- val serializedBytes = Utils.toArray(RequestTestUtils.serializeRequestWithHeader(header, request))
+ private def sendWithHeader(request: AbstractRequest, header: RequestHeader, socket: Socket): Unit = {
+ val serializedBytes = Utils.toArray(request.serializeWithHeader(header))
sendRequest(socket, serializedBytes)
}
diff --git a/core/src/test/scala/unit/kafka/network/RequestChannelTest.scala b/core/src/test/scala/unit/kafka/network/RequestChannelTest.scala
index 49e2401..78618c2 100644
--- a/core/src/test/scala/unit/kafka/network/RequestChannelTest.scala
+++ b/core/src/test/scala/unit/kafka/network/RequestChannelTest.scala
@@ -32,8 +32,8 @@ import org.apache.kafka.common.memory.MemoryPool
import org.apache.kafka.common.message.IncrementalAlterConfigsRequestData
import org.apache.kafka.common.message.IncrementalAlterConfigsRequestData._
import org.apache.kafka.common.network.{ClientInformation, ListenerName}
-import org.apache.kafka.common.requests._
import org.apache.kafka.common.requests.AlterConfigsRequest._
+import org.apache.kafka.common.requests._
import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol}
import org.easymock.EasyMock._
import org.junit.jupiter.api.Assertions._
@@ -190,8 +190,7 @@ class RequestChannelTest {
}
def request(req: AbstractRequest): RequestChannel.Request = {
- val buffer = RequestTestUtils.serializeRequestWithHeader(new RequestHeader(req.apiKey, req.version, "client-id", 1),
- req)
+ val buffer = req.serializeWithHeader(new RequestHeader(req.apiKey, req.version, "client-id", 1))
val requestContext = newRequestContext(buffer)
new network.RequestChannel.Request(processor = 1,
requestContext,
diff --git a/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala
index e74e77d..9b8db57 100644
--- a/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala
+++ b/core/src/test/scala/unit/kafka/network/RequestConvertToJsonTest.scala
@@ -26,12 +26,12 @@ import kafka.network.RequestConvertToJson.requestHeaderNode
import org.apache.kafka.common.memory.MemoryPool
import org.apache.kafka.common.message._
import org.apache.kafka.common.network.{ClientInformation, ListenerName, NetworkSend}
-import org.junit.jupiter.api.Test
-import org.apache.kafka.common.protocol.{ApiKeys, ByteBufferAccessor, ObjectSerializationCache}
+import org.apache.kafka.common.protocol.{ApiKeys, MessageUtil}
import org.apache.kafka.common.requests._
import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol}
import org.easymock.EasyMock.createNiceMock
import org.junit.jupiter.api.Assertions.assertEquals
+import org.junit.jupiter.api.Test
import scala.collection.mutable.ArrayBuffer
@@ -42,7 +42,6 @@ class RequestConvertToJsonTest {
val unhandledKeys = ArrayBuffer[String]()
ApiKeys.values().foreach { key => {
val version: Short = key.latestVersion()
- val cache = new ObjectSerializationCache
val message = key match {
case ApiKeys.DESCRIBE_ACLS =>
ApiMessageType.fromApiKey(key.id).newRequest().asInstanceOf[DescribeAclsRequestData]
@@ -50,12 +49,9 @@ class RequestConvertToJsonTest {
case _ =>
ApiMessageType.fromApiKey(key.id).newRequest()
}
- val messageSize = message.size(cache, version)
- val bytes = new ByteBufferAccessor(ByteBuffer.allocate(messageSize))
- message.write(bytes, cache, version)
- bytes.flip()
- val req = AbstractRequest.parseRequest(key, version, bytes.buffer).request
+ val bytes = MessageUtil.toByteBuffer(message, version)
+ val req = AbstractRequest.parseRequest(key, version, bytes).request
try {
RequestConvertToJson.request(req)
} catch {
@@ -70,13 +66,9 @@ class RequestConvertToJsonTest {
val unhandledKeys = ArrayBuffer[String]()
ApiKeys.values().foreach { key => {
val version: Short = key.latestVersion()
- val cache = new ObjectSerializationCache
val message = ApiMessageType.fromApiKey(key.id).newResponse()
- val messageSize = message.size(cache, version)
- val bytes = new ByteBufferAccessor(ByteBuffer.allocate(messageSize))
- message.write(bytes, cache, version)
- bytes.flip()
- val res = AbstractResponse.parseResponse(key, bytes.buffer, version)
+ val bytes = MessageUtil.toByteBuffer(message, version)
+ val res = AbstractResponse.parseResponse(key, bytes, version)
try {
RequestConvertToJson.response(res, version)
} catch {
@@ -171,8 +163,7 @@ class RequestConvertToJsonTest {
}
def request(req: AbstractRequest): RequestChannel.Request = {
- val buffer = RequestTestUtils.serializeRequestWithHeader(new RequestHeader(req.apiKey, req.version, "client-id", 1),
- req)
+ val buffer = req.serializeWithHeader(new RequestHeader(req.apiKey, req.version, "client-id", 1))
val requestContext = newRequestContext(buffer)
new network.RequestChannel.Request(processor = 1,
requestContext,
diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala
index 52aaeef..01e7f20 100644
--- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala
+++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala
@@ -114,7 +114,7 @@ class SocketServerTest {
}
def sendApiRequest(socket: Socket, request: AbstractRequest, header: RequestHeader): Unit = {
- val serializedBytes = Utils.toArray(RequestTestUtils.serializeRequestWithHeader(header, request))
+ val serializedBytes = Utils.toArray(request.serializeWithHeader(header))
sendRequest(socket, serializedBytes)
}
@@ -140,7 +140,7 @@ class SocketServerTest {
}
def processRequest(channel: RequestChannel, request: RequestChannel.Request): Unit = {
- val byteBuffer = RequestTestUtils.serializeRequestWithHeader(request.header, request.body[AbstractRequest])
+ val byteBuffer = request.body[AbstractRequest].serializeWithHeader(request.header)
val send = new NetworkSend(request.context.connectionId, ByteBufferSend.sizePrefixed(byteBuffer))
val headerLog = RequestConvertToJson.requestHeaderNode(request.header)
channel.sendResponse(new RequestChannel.SendResponse(request, send, Some(headerLog), None))
@@ -212,13 +212,13 @@ class SocketServerTest {
.setTransactionalId(null))
.build()
val emptyHeader = new RequestHeader(ApiKeys.PRODUCE, emptyRequest.version, clientId, correlationId)
- Utils.toArray(RequestTestUtils.serializeRequestWithHeader(emptyHeader, emptyRequest))
+ Utils.toArray(emptyRequest.serializeWithHeader(emptyHeader))
}
private def apiVersionRequestBytes(clientId: String, version: Short): Array[Byte] = {
val request = new ApiVersionsRequest.Builder().build(version)
val header = new RequestHeader(ApiKeys.API_VERSIONS, request.version(), clientId, -1)
- Utils.toArray(RequestTestUtils.serializeRequestWithHeader(header, request))
+ Utils.toArray(request.serializeWithHeader(header))
}
@Test
@@ -367,7 +367,7 @@ class SocketServerTest {
val correlationId = 57
val header = new RequestHeader(ApiKeys.VOTE, 0, "", correlationId)
val request = new VoteRequest.Builder(new VoteRequestData()).build()
- val serializedBytes = Utils.toArray(RequestTestUtils.serializeRequestWithHeader(header, request))
+ val serializedBytes = Utils.toArray(request.serializeWithHeader(header))
val socket = connect()
@@ -676,7 +676,7 @@ class SocketServerTest {
// Mimic a primitive request handler that fetches the request from RequestChannel and place a response with a
// throttled channel.
val request = receiveRequest(server.dataPlaneRequestChannel)
- val byteBuffer = RequestTestUtils.serializeRequestWithHeader(request.header, request.body[AbstractRequest])
+ val byteBuffer = request.body[AbstractRequest].serializeWithHeader(request.header)
val send = new NetworkSend(request.context.connectionId, ByteBufferSend.sizePrefixed(byteBuffer))
val channelThrottlingCallback = new ThrottleCallback {
@@ -971,7 +971,7 @@ class SocketServerTest {
.setTransactionalId(null))
.build()
val emptyHeader = new RequestHeader(ApiKeys.PRODUCE, emptyRequest.version, clientId, correlationId)
- val serializedBytes = Utils.toArray(RequestTestUtils.serializeRequestWithHeader(emptyHeader, emptyRequest))
+ val serializedBytes = Utils.toArray(emptyRequest.serializeWithHeader(emptyHeader))
sendRequest(sslSocket, serializedBytes)
processRequest(overrideServer.dataPlaneRequestChannel)
diff --git a/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala
index 6a2baca..f086443 100644
--- a/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala
@@ -16,16 +16,16 @@
*/
package kafka.server
-import java.util.Properties
-
import kafka.test.ClusterInstance
import org.apache.kafka.common.message.ApiMessageType.ListenerType
import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersion
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.protocol.ApiKeys
-import org.apache.kafka.common.requests.{ApiVersionsRequest, ApiVersionsResponse}
+import org.apache.kafka.common.requests.{ApiVersionsRequest, ApiVersionsResponse, RequestUtils}
+import org.apache.kafka.common.utils.Utils
import org.junit.jupiter.api.Assertions._
+import java.util.Properties
import scala.jdk.CollectionConverters._
abstract class AbstractApiVersionsRequestTest(cluster: ClusterInstance) {
@@ -49,7 +49,9 @@ abstract class AbstractApiVersionsRequestTest(cluster: ClusterInstance) {
val overrideHeader = IntegrationTestUtils.nextRequestHeader(ApiKeys.API_VERSIONS, Short.MaxValue)
val socket = IntegrationTestUtils.connect(cluster.brokerSocketServers().asScala.head, cluster.clientListener())
try {
- IntegrationTestUtils.sendWithHeader(request, overrideHeader, socket)
+ val serializedBytes = Utils.toArray(
+ RequestUtils.serialize(overrideHeader.data, overrideHeader.headerVersion, request.data, request.version))
+ IntegrationTestUtils.sendRequest(socket, serializedBytes)
IntegrationTestUtils.receive[ApiVersionsResponse](socket, ApiKeys.API_VERSIONS, 0.toShort)
} finally socket.close()
}
diff --git a/core/src/test/scala/unit/kafka/server/BaseClientQuotaManagerTest.scala b/core/src/test/scala/unit/kafka/server/BaseClientQuotaManagerTest.scala
index 48379ca..1fe01bf 100644
--- a/core/src/test/scala/unit/kafka/server/BaseClientQuotaManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/BaseClientQuotaManagerTest.scala
@@ -19,18 +19,16 @@ package kafka.server
import java.net.InetAddress
import java.util
import java.util.Collections
+
import kafka.network.RequestChannel
import kafka.network.RequestChannel.Session
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.memory.MemoryPool
-import org.apache.kafka.common.metrics.MetricConfig
-import org.apache.kafka.common.metrics.Metrics
-import org.apache.kafka.common.network.ClientInformation
-import org.apache.kafka.common.network.ListenerName
-import org.apache.kafka.common.requests.{AbstractRequest, FetchRequest, RequestContext, RequestHeader, RequestTestUtils}
+import org.apache.kafka.common.metrics.{MetricConfig, Metrics}
+import org.apache.kafka.common.network.{ClientInformation, ListenerName}
import org.apache.kafka.common.requests.FetchRequest.PartitionData
-import org.apache.kafka.common.security.auth.KafkaPrincipal
-import org.apache.kafka.common.security.auth.SecurityProtocol
+import org.apache.kafka.common.requests.{AbstractRequest, FetchRequest, RequestContext, RequestHeader}
+import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol}
import org.apache.kafka.common.utils.MockTime
import org.easymock.EasyMock
import org.junit.jupiter.api.AfterEach
@@ -57,8 +55,7 @@ class BaseClientQuotaManagerTest {
listenerName: ListenerName = ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT)): (T, RequestChannel.Request) = {
val request = builder.build()
- val buffer = RequestTestUtils.serializeRequestWithHeader(
- new RequestHeader(builder.apiKey, request.version, "", 0), request)
+ val buffer = request.serializeWithHeader(new RequestHeader(builder.apiKey, request.version, "", 0))
val requestChannelMetrics: RequestChannel.Metrics = EasyMock.createNiceMock(classOf[RequestChannel.Metrics])
// read the header from the buffer first so that the body can be read next from the Request constructor
diff --git a/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala
index 7b51bfe..3d3d0ca 100644
--- a/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala
@@ -17,20 +17,19 @@
package kafka.server
-import java.io.{DataInputStream, DataOutputStream}
-import java.net.Socket
-import java.nio.ByteBuffer
-import java.util.Properties
-
import kafka.api.IntegrationTestHarness
import kafka.network.SocketServer
import kafka.utils.NotNothing
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.protocol.ApiKeys
-import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, RequestHeader, RequestTestUtils, ResponseHeader}
+import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, RequestHeader, ResponseHeader}
import org.apache.kafka.common.utils.Utils
import org.apache.kafka.metadata.BrokerState
+import java.io.{DataInputStream, DataOutputStream}
+import java.net.Socket
+import java.nio.ByteBuffer
+import java.util.Properties
import scala.annotation.nowarn
import scala.collection.Seq
import scala.reflect.ClassTag
@@ -136,7 +135,7 @@ abstract class BaseRequestTest extends IntegrationTestHarness {
}
def sendWithHeader(request: AbstractRequest, header: RequestHeader, socket: Socket): Unit = {
- val serializedBytes = Utils.toArray(RequestTestUtils.serializeRequestWithHeader(header, request))
+ val serializedBytes = Utils.toArray(request.serializeWithHeader(header))
sendRequest(socket, serializedBytes)
}
diff --git a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala
index 44d82e4..3ed4faa 100644
--- a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala
@@ -121,8 +121,7 @@ class ControllerApisTest {
request: AbstractRequest,
listenerName: ListenerName = ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT)
): RequestChannel.Request = {
- val buffer = RequestTestUtils.serializeRequestWithHeader(
- new RequestHeader(request.apiKey, request.version, clientID, 0), request)
+ val buffer = request.serializeWithHeader(new RequestHeader(request.apiKey, request.version, clientID, 0))
// read the header from the buffer first so that the body can be read next from the Request constructor
val header = RequestHeader.parse(buffer)
diff --git a/core/src/test/scala/unit/kafka/server/ForwardingManagerTest.scala b/core/src/test/scala/unit/kafka/server/ForwardingManagerTest.scala
index 2fefdac..d0fc30f 100644
--- a/core/src/test/scala/unit/kafka/server/ForwardingManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ForwardingManagerTest.scala
@@ -207,7 +207,7 @@ class ForwardingManagerTest {
"clientId",
correlationId
)
- val buffer = RequestTestUtils.serializeRequestWithHeader(header, body)
+ val buffer = body.serializeWithHeader(header)
// Fast-forward buffer to start of the request as `RequestChannel.Request` expects
RequestHeader.parse(buffer)
diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
index bd0de39..1f6e013 100644
--- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
+++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
@@ -3109,14 +3109,14 @@ class KafkaApisTest {
val listenerName = ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT)
val requestHeader = new RequestHeader(request.apiKey, request.version, clientId, 0)
- val requestBuffer = RequestTestUtils.serializeRequestWithHeader(requestHeader, request)
+ val requestBuffer = request.serializeWithHeader(requestHeader)
val envelopeHeader = new RequestHeader(ApiKeys.ENVELOPE, ApiKeys.ENVELOPE.latestVersion(), clientId, 0)
- val envelopeBuffer = RequestTestUtils.serializeRequestWithHeader(envelopeHeader, new EnvelopeRequest.Builder(
+ val envelopeBuffer = new EnvelopeRequest.Builder(
requestBuffer,
principalSerde.serialize(KafkaPrincipal.ANONYMOUS),
InetAddress.getLocalHost.getAddress
- ).build())
+ ).build().serializeWithHeader(envelopeHeader)
val envelopeContext = new RequestContext(envelopeHeader, "1", InetAddress.getLocalHost,
KafkaPrincipal.ANONYMOUS, listenerName, SecurityProtocol.PLAINTEXT, ClientInformation.EMPTY,
fromPrivilegedListener, Optional.of(principalSerde))
@@ -3136,8 +3136,8 @@ class KafkaApisTest {
listenerName: ListenerName = ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT),
fromPrivilegedListener: Boolean = false,
requestHeader: Option[RequestHeader] = None): RequestChannel.Request = {
- val buffer = RequestTestUtils.serializeRequestWithHeader(requestHeader.getOrElse(
- new RequestHeader(request.apiKey, request.version, clientId, 0)), request)
+ val buffer = request.serializeWithHeader(
+ requestHeader.getOrElse(new RequestHeader(request.apiKey, request.version, clientId, 0)))
// read the header from the buffer first so that the body can be read next from the Request constructor
val header = RequestHeader.parse(buffer)
diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala
index b992aac..35f6ff8 100644
--- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala
+++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala
@@ -612,8 +612,8 @@ class RequestQuotaTest extends BaseRequestTest {
"client-id",
0
)
- val embedRequestData = RequestTestUtils.serializeRequestWithHeader(requestHeader,
- new AlterClientQuotasRequest.Builder(List.empty.asJava, false).build())
+ val embedRequestData = new AlterClientQuotasRequest.Builder(List.empty.asJava, false).build()
+ .serializeWithHeader(requestHeader)
new EnvelopeRequest.Builder(embedRequestData, new Array[Byte](0),
InetAddress.getByName("192.168.1.1").getAddress)