Merge pull request #21904 from lidizheng/aio-benchmark-client

[Aio] Benchmark automation
pull/22093/head
Lidi Zheng 5 years ago committed by GitHub
commit 2eeb22de65
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 11
      src/proto/grpc/core/BUILD
  2. 52
      src/proto/grpc/testing/BUILD
  3. 6
      src/proto/grpc/testing/control.proto
  4. 2
      src/python/grpcio/grpc/_cython/_cygrpc/aio/common.pyx.pxi
  5. 5
      src/python/grpcio/grpc/_cython/_cygrpc/aio/server.pyx.pxi
  6. 27
      src/python/grpcio_tests/tests/qps/BUILD.bazel
  7. 10
      src/python/grpcio_tests/tests/qps/histogram.py
  8. 63
      src/python/grpcio_tests/tests_aio/benchmark/BUILD.bazel
  9. 155
      src/python/grpcio_tests/tests_aio/benchmark/benchmark_client.py
  10. 55
      src/python/grpcio_tests/tests_aio/benchmark/benchmark_servicer.py
  11. 18
      src/python/grpcio_tests/tests_aio/benchmark/server.py
  12. 58
      src/python/grpcio_tests/tests_aio/benchmark/worker.py
  13. 367
      src/python/grpcio_tests/tests_aio/benchmark/worker_servicer.py
  14. 2
      test/cpp/qps/json_run_localhost_scenarios.bzl
  15. 2
      test/cpp/qps/qps_json_driver_scenarios.bzl
  16. 2
      tools/internal_ci/linux/grpc_e2e_performance_singlevm.sh
  17. 2
      tools/internal_ci/linux/grpc_full_performance_master.sh
  18. 2
      tools/internal_ci/linux/grpc_full_performance_release.sh
  19. 368
      tools/run_tests/generated/tests.json
  20. 4
      tools/run_tests/performance/build_performance.sh
  21. 20
      tools/run_tests/performance/run_worker_python_asyncio.sh
  22. 157
      tools/run_tests/performance/scenario_config.py

@ -15,6 +15,7 @@
licenses(["notice"]) # Apache v2
load("//bazel:grpc_build_system.bzl", "grpc_package", "grpc_proto_library")
load("//bazel:python_rules.bzl", "py_proto_library")
grpc_package(
name = "core",
@ -25,3 +26,13 @@ grpc_proto_library(
name = "stats_proto",
srcs = ["stats.proto"],
)
proto_library(
name = "stats_descriptor",
srcs = ["stats.proto"],
)
py_proto_library(
name = "stats_py_pb2",
deps = [":stats_descriptor"],
)

@ -233,3 +233,55 @@ py_grpc_library(
srcs = [":test_proto_descriptor"],
deps = [":py_test_proto"],
)
proto_library(
name = "worker_service_descriptor",
srcs = ["worker_service.proto"],
deps = [":control_descriptor"],
)
py_proto_library(
name = "worker_service_py_pb2",
deps = [":worker_service_descriptor"],
)
py_grpc_library(
name = "worker_service_py_pb2_grpc",
srcs = [":worker_service_descriptor"],
deps = [":worker_service_py_pb2"],
)
proto_library(
name = "stats_descriptor",
srcs = ["stats.proto"],
deps = ["//src/proto/grpc/core:stats_descriptor"],
)
py_proto_library(
name = "stats_py_pb2",
deps = [":stats_descriptor"],
)
proto_library(
name = "payloads_descriptor",
srcs = ["payloads.proto"],
)
py_proto_library(
name = "payloads_py_pb2",
deps = [":payloads_descriptor"],
)
proto_library(
name = "control_descriptor",
srcs = ["control.proto"],
deps = [
":payloads_descriptor",
":stats_descriptor",
],
)
py_proto_library(
name = "control_py_pb2",
deps = [":control_descriptor"],
)

@ -117,6 +117,9 @@ message ClientConfig {
// If 0, disabled. Else, specifies the period between gathering latency
// medians in milliseconds.
int32 median_latency_collection_interval_millis = 20;
// Number of client processes. 0 indicates no restriction.
int32 client_processes = 21;
}
message ClientStatus { ClientStats stats = 1; }
@ -163,6 +166,9 @@ message ServerConfig {
// Buffer pool size (no buffer pool specified if unset)
int32 resource_quota_size = 1001;
repeated ChannelArg channel_args = 1002;
// Number of server processes. 0 indicates no restriction.
int32 server_processes = 21;
}
message ServerArgs {

@ -42,6 +42,8 @@ cdef bytes serialize(object serializer, object message):
Failure to serialize is a fatal error.
"""
if isinstance(message, str):
message = message.encode('utf-8')
if serializer:
return serializer(message)
else:

@ -333,6 +333,9 @@ async def _handle_unary_unary_rpc(object method_handler,
object loop):
# Receives request message
cdef bytes request_raw = await _receive_message(rpc_state, loop)
if request_raw is None:
# The RPC was cancelled immediately after start on client side.
return
# Deserializes the request message
cdef object request_message = deserialize(
@ -364,6 +367,8 @@ async def _handle_unary_stream_rpc(object method_handler,
object loop):
# Receives request message
cdef bytes request_raw = await _receive_message(rpc_state, loop)
if request_raw is None:
return
# Deserializes the request message
cdef object request_message = deserialize(

@ -0,0 +1,27 @@
# Copyright 2020 The gRPC Authors
#
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
# You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
package(
default_testonly = 1,
default_visibility = ["//visibility:public"],
)
py_library(
name = "histogram",
srcs = ["histogram.py"],
srcs_version = "PY2AND3",
deps = [
"//src/proto/grpc/testing:stats_py_pb2",
],
)

@ -65,6 +65,16 @@ class Histogram(object):
data.count = self._count
return data
def merge(self, another_data):
with self._lock:
for i in range(len(self._buckets)):
self._buckets[i] += another_data.bucket[i]
self._min = min(self._min, another_data.min_seen)
self._max = max(self._max, another_data.max_seen)
self._sum += another_data.sum
self._sum_of_squares += another_data.sum_of_squares
self._count += another_data.count
def _bucket_for(self, val):
val = min(val, self._max_possible)
return int(math.log(val, self.multiplier))

@ -17,16 +17,67 @@ package(
default_visibility = ["//visibility:public"],
)
py_binary(
name = "server",
srcs = ["server.py"],
python_version = "PY3",
py_library(
name = "benchmark_client",
srcs = ["benchmark_client.py"],
srcs_version = "PY3",
deps = [
"//src/proto/grpc/testing:benchmark_service_py_pb2",
"//src/proto/grpc/testing:benchmark_service_py_pb2_grpc",
"//src/proto/grpc/testing:py_messages_proto",
"//src/python/grpcio/grpc:grpcio",
"//src/python/grpcio_tests/tests/qps:histogram",
"//src/python/grpcio_tests/tests/unit:resources",
],
)
py_library(
name = "benchmark_servicer",
srcs = ["benchmark_servicer.py"],
srcs_version = "PY3",
deps = [
"//src/proto/grpc/testing:benchmark_service_py_pb2_grpc",
"//src/proto/grpc/testing:py_messages_proto",
"//src/python/grpcio/grpc:grpcio",
],
)
py_library(
name = "worker_servicer",
srcs = ["worker_servicer.py"],
data = [
"//src/python/grpcio_tests/tests/unit/credentials",
],
srcs_version = "PY3",
deps = [
":benchmark_client",
":benchmark_servicer",
"//src/proto/grpc/core:stats_py_pb2",
"//src/proto/grpc/testing:benchmark_service_py_pb2_grpc",
"//src/proto/grpc/testing:control_py_pb2",
"//src/proto/grpc/testing:payloads_py_pb2",
"//src/proto/grpc/testing:stats_py_pb2",
"//src/proto/grpc/testing:worker_service_py_pb2_grpc",
"//src/python/grpcio/grpc:grpcio",
"//src/python/grpcio_tests/tests/qps:histogram",
"//src/python/grpcio_tests/tests/unit:resources",
"//src/python/grpcio_tests/tests/unit/framework/common",
"@six",
],
)
py_binary(
name = "server",
srcs = ["server.py"],
python_version = "PY3",
deps = [":benchmark_servicer"],
)
py_binary(
name = "worker",
srcs = ["worker.py"],
imports = ["../../"],
python_version = "PY3",
deps = [
":worker_servicer",
"//src/proto/grpc/testing:worker_service_py_pb2_grpc",
],
)

@ -0,0 +1,155 @@
# Copyright 2020 The gRPC Authors
#
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
# You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
"""The Python AsyncIO Benchmark Clients."""
import abc
import asyncio
import time
import logging
import random
import grpc
from grpc.experimental import aio
from src.proto.grpc.testing import (benchmark_service_pb2_grpc, control_pb2,
messages_pb2)
from tests.qps import histogram
from tests.unit import resources
class GenericStub(object):
def __init__(self, channel: aio.Channel):
self.UnaryCall = channel.unary_unary(
'/grpc.testing.BenchmarkService/UnaryCall')
self.StreamingCall = channel.stream_stream(
'/grpc.testing.BenchmarkService/StreamingCall')
class BenchmarkClient(abc.ABC):
"""Benchmark client interface that exposes a non-blocking send_request()."""
def __init__(self, address: str, config: control_pb2.ClientConfig,
hist: histogram.Histogram):
# Disables underlying reuse of subchannels
unique_option = (('iv', random.random()),)
# Parses the channel argument from config
channel_args = tuple(
(arg.name, arg.str_value) if arg.HasField('str_value') else (
arg.name, int(arg.int_value)) for arg in config.channel_args)
# Creates the channel
if config.HasField('security_params'):
channel_credentials = grpc.ssl_channel_credentials(
resources.test_root_certificates(),)
server_host_override_option = ((
'grpc.ssl_target_name_override',
config.security_params.server_host_override,
),)
self._channel = aio.secure_channel(
address, channel_credentials,
unique_option + channel_args + server_host_override_option)
else:
self._channel = aio.insecure_channel(address,
options=unique_option +
channel_args)
# Creates the stub
if config.payload_config.WhichOneof('payload') == 'simple_params':
self._generic = False
self._stub = benchmark_service_pb2_grpc.BenchmarkServiceStub(
self._channel)
payload = messages_pb2.Payload(
body=b'\0' * config.payload_config.simple_params.req_size)
self._request = messages_pb2.SimpleRequest(
payload=payload,
response_size=config.payload_config.simple_params.resp_size)
else:
self._generic = True
self._stub = GenericStub(self._channel)
self._request = b'\0' * config.payload_config.bytebuf_params.req_size
self._hist = hist
self._response_callbacks = []
self._concurrency = config.outstanding_rpcs_per_channel
async def run(self) -> None:
await self._channel.channel_ready()
async def stop(self) -> None:
await self._channel.close()
def _record_query_time(self, query_time: float) -> None:
self._hist.add(query_time * 1e9)
class UnaryAsyncBenchmarkClient(BenchmarkClient):
def __init__(self, address: str, config: control_pb2.ClientConfig,
hist: histogram.Histogram):
super().__init__(address, config, hist)
self._running = None
self._stopped = asyncio.Event()
async def _send_request(self):
start_time = time.monotonic()
await self._stub.UnaryCall(self._request)
self._record_query_time(time.monotonic() - start_time)
async def _send_indefinitely(self) -> None:
while self._running:
await self._send_request()
async def run(self) -> None:
await super().run()
self._running = True
senders = (self._send_indefinitely() for _ in range(self._concurrency))
await asyncio.gather(*senders)
self._stopped.set()
async def stop(self) -> None:
self._running = False
await self._stopped.wait()
await super().stop()
class StreamingAsyncBenchmarkClient(BenchmarkClient):
def __init__(self, address: str, config: control_pb2.ClientConfig,
hist: histogram.Histogram):
super().__init__(address, config, hist)
self._running = None
self._stopped = asyncio.Event()
async def _one_streaming_call(self):
call = self._stub.StreamingCall()
while self._running:
start_time = time.time()
await call.write(self._request)
await call.read()
self._record_query_time(time.time() - start_time)
await call.done_writing()
async def run(self):
await super().run()
self._running = True
senders = (self._one_streaming_call() for _ in range(self._concurrency))
await asyncio.gather(*senders)
self._stopped.set()
async def stop(self):
self._running = False
await self._stopped.wait()
await super().stop()

@ -0,0 +1,55 @@
# Copyright 2020 The gRPC Authors
#
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
# You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
"""The Python AsyncIO Benchmark Servicers."""
import asyncio
import logging
import unittest
from grpc.experimental import aio
from src.proto.grpc.testing import benchmark_service_pb2_grpc, messages_pb2
class BenchmarkServicer(benchmark_service_pb2_grpc.BenchmarkServiceServicer):
async def UnaryCall(self, request, unused_context):
payload = messages_pb2.Payload(body=b'\0' * request.response_size)
return messages_pb2.SimpleResponse(payload=payload)
async def StreamingFromServer(self, request, unused_context):
payload = messages_pb2.Payload(body=b'\0' * request.response_size)
# Sends response at full capacity!
while True:
yield messages_pb2.SimpleResponse(payload=payload)
async def StreamingCall(self, request_iterator, unused_context):
async for request in request_iterator:
payload = messages_pb2.Payload(body=b'\0' * request.response_size)
yield messages_pb2.SimpleResponse(payload=payload)
class GenericBenchmarkServicer(
benchmark_service_pb2_grpc.BenchmarkServiceServicer):
"""Generic (no-codec) Server implementation for the Benchmark service."""
def __init__(self, resp_size):
self._response = '\0' * resp_size
async def UnaryCall(self, unused_request, unused_context):
return self._response
async def StreamingCall(self, request_iterator, unused_context):
async for _ in request_iterator:
yield self._response

@ -17,28 +17,16 @@ import logging
import unittest
from grpc.experimental import aio
from src.proto.grpc.testing import messages_pb2
from src.proto.grpc.testing import benchmark_service_pb2_grpc
class BenchmarkServer(benchmark_service_pb2_grpc.BenchmarkServiceServicer):
async def UnaryCall(self, request, context):
payload = messages_pb2.Payload(body=b'\0' * request.response_size)
return messages_pb2.SimpleResponse(payload=payload)
async def StreamingFromServer(self, request, context):
payload = messages_pb2.Payload(body=b'\0' * request.response_size)
# Sends response at full capacity!
while True:
yield messages_pb2.SimpleResponse(payload=payload)
from src.proto.grpc.testing import benchmark_service_pb2_grpc
from tests_aio.benchmark import benchmark_servicer
async def _start_async_server():
server = aio.server()
port = server.add_insecure_port('localhost:%s' % 50051)
servicer = BenchmarkServer()
servicer = benchmark_servicer.BenchmarkServicer()
benchmark_service_pb2_grpc.add_BenchmarkServiceServicer_to_server(
servicer, server)

@ -0,0 +1,58 @@
# Copyright 2020 The gRPC Authors
#
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
# You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
import argparse
import asyncio
import logging
from grpc.experimental import aio
from src.proto.grpc.testing import worker_service_pb2_grpc
from tests_aio.benchmark import worker_servicer
async def run_worker_server(port: int) -> None:
aio.init_grpc_aio()
server = aio.server()
servicer = worker_servicer.WorkerServicer()
worker_service_pb2_grpc.add_WorkerServiceServicer_to_server(
servicer, server)
server.add_insecure_port('[::]:{}'.format(port))
await server.start()
await servicer.wait_for_quit()
await server.stop(None)
if __name__ == '__main__':
logging.basicConfig(level=logging.DEBUG)
parser = argparse.ArgumentParser(
description='gRPC Python performance testing worker')
parser.add_argument('--driver_port',
type=int,
dest='port',
help='The port the worker should listen on')
parser.add_argument('--uvloop',
action='store_true',
help='Use uvloop or not')
args = parser.parse_args()
if args.uvloop:
import uvloop
uvloop.install()
asyncio.get_event_loop().run_until_complete(run_worker_server(args.port))

@ -0,0 +1,367 @@
# Copyright 2020 The gRPC Authors
#
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
# You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
import asyncio
import collections
import logging
import multiprocessing
import os
import sys
import time
from typing import Tuple
import grpc
from grpc.experimental import aio
from src.proto.grpc.testing import (benchmark_service_pb2_grpc, control_pb2,
stats_pb2, worker_service_pb2_grpc)
from tests.qps import histogram
from tests.unit import resources
from tests.unit.framework.common import get_socket
from tests_aio.benchmark import benchmark_client, benchmark_servicer
_NUM_CORES = multiprocessing.cpu_count()
_WORKER_ENTRY_FILE = os.path.join(
os.path.split(os.path.abspath(__file__))[0], 'worker.py')
_LOGGER = logging.getLogger(__name__)
class _SubWorker(
collections.namedtuple('_SubWorker',
['process', 'port', 'channel', 'stub'])):
"""A data class that holds information about a child qps worker."""
def _repr(self):
return f'<_SubWorker pid={self.process.pid} port={self.port}>'
def __repr__(self):
return self._repr()
def __str__(self):
return self._repr()
def _get_server_status(start_time: float, end_time: float,
port: int) -> control_pb2.ServerStatus:
"""Creates ServerStatus proto message."""
end_time = time.monotonic()
elapsed_time = end_time - start_time
# TODO(lidiz) Collect accurate time system to compute QPS/core-second.
stats = stats_pb2.ServerStats(time_elapsed=elapsed_time,
time_user=elapsed_time,
time_system=elapsed_time)
return control_pb2.ServerStatus(stats=stats, port=port, cores=_NUM_CORES)
def _create_server(config: control_pb2.ServerConfig) -> Tuple[aio.Server, int]:
"""Creates a server object according to the ServerConfig."""
channel_args = tuple(
(arg.name,
arg.str_value) if arg.HasField('str_value') else (arg.name,
int(arg.int_value))
for arg in config.channel_args)
server = aio.server(options=channel_args + (('grpc.so_reuseport', 1),))
if config.server_type == control_pb2.ASYNC_SERVER:
servicer = benchmark_servicer.BenchmarkServicer()
benchmark_service_pb2_grpc.add_BenchmarkServiceServicer_to_server(
servicer, server)
elif config.server_type == control_pb2.ASYNC_GENERIC_SERVER:
resp_size = config.payload_config.bytebuf_params.resp_size
servicer = benchmark_servicer.GenericBenchmarkServicer(resp_size)
method_implementations = {
'StreamingCall':
grpc.stream_stream_rpc_method_handler(servicer.StreamingCall),
'UnaryCall':
grpc.unary_unary_rpc_method_handler(servicer.UnaryCall),
}
handler = grpc.method_handlers_generic_handler(
'grpc.testing.BenchmarkService', method_implementations)
server.add_generic_rpc_handlers((handler,))
else:
raise NotImplementedError('Unsupported server type {}'.format(
config.server_type))
if config.HasField('security_params'): # Use SSL
server_creds = grpc.ssl_server_credentials(
((resources.private_key(), resources.certificate_chain()),))
port = server.add_secure_port('[::]:{}'.format(config.port),
server_creds)
else:
port = server.add_insecure_port('[::]:{}'.format(config.port))
return server, port
def _get_client_status(start_time: float, end_time: float,
qps_data: histogram.Histogram
) -> control_pb2.ClientStatus:
"""Creates ClientStatus proto message."""
latencies = qps_data.get_data()
end_time = time.monotonic()
elapsed_time = end_time - start_time
# TODO(lidiz) Collect accurate time system to compute QPS/core-second.
stats = stats_pb2.ClientStats(latencies=latencies,
time_elapsed=elapsed_time,
time_user=elapsed_time,
time_system=elapsed_time)
return control_pb2.ClientStatus(stats=stats)
def _create_client(server: str, config: control_pb2.ClientConfig,
qps_data: histogram.Histogram
) -> benchmark_client.BenchmarkClient:
"""Creates a client object according to the ClientConfig."""
if config.load_params.WhichOneof('load') != 'closed_loop':
raise NotImplementedError(
f'Unsupported load parameter {config.load_params}')
if config.client_type == control_pb2.ASYNC_CLIENT:
if config.rpc_type == control_pb2.UNARY:
client_type = benchmark_client.UnaryAsyncBenchmarkClient
elif config.rpc_type == control_pb2.STREAMING:
client_type = benchmark_client.StreamingAsyncBenchmarkClient
else:
raise NotImplementedError(
f'Unsupported rpc_type [{config.rpc_type}]')
else:
raise NotImplementedError(
f'Unsupported client type {config.client_type}')
return client_type(server, config, qps_data)
def _pick_an_unused_port() -> int:
"""Picks an unused TCP port."""
_, port, sock = get_socket()
sock.close()
return port
async def _create_sub_worker() -> _SubWorker:
"""Creates a child qps worker as a subprocess."""
port = _pick_an_unused_port()
_LOGGER.info('Creating sub worker at port [%d]...', port)
process = await asyncio.create_subprocess_exec(sys.executable,
_WORKER_ENTRY_FILE,
'--driver_port', str(port))
_LOGGER.info('Created sub worker process for port [%d] at pid [%d]', port,
process.pid)
channel = aio.insecure_channel(f'localhost:{port}')
_LOGGER.info('Waiting for sub worker at port [%d]', port)
await channel.channel_ready()
stub = worker_service_pb2_grpc.WorkerServiceStub(channel)
return _SubWorker(
process=process,
port=port,
channel=channel,
stub=stub,
)
class WorkerServicer(worker_service_pb2_grpc.WorkerServiceServicer):
"""Python Worker Server implementation."""
def __init__(self):
self._loop = asyncio.get_event_loop()
self._quit_event = asyncio.Event()
async def _run_single_server(self, config, request_iterator, context):
server, port = _create_server(config)
await server.start()
_LOGGER.info('Server started at port [%d]', port)
start_time = time.monotonic()
await context.write(_get_server_status(start_time, start_time, port))
async for request in request_iterator:
end_time = time.monotonic()
status = _get_server_status(start_time, end_time, port)
if request.mark.reset:
start_time = end_time
await context.write(status)
await server.stop(None)
async def RunServer(self, request_iterator, context):
config_request = await context.read()
config = config_request.setup
_LOGGER.info('Received ServerConfig: %s', config)
if config.server_processes <= 0:
_LOGGER.info('Using server_processes == [%d]', _NUM_CORES)
config.server_processes = _NUM_CORES
if config.port == 0:
config.port = _pick_an_unused_port()
_LOGGER.info('Port picked [%d]', config.port)
if config.server_processes == 1:
# If server_processes == 1, start the server in this process.
await self._run_single_server(config, request_iterator, context)
else:
# If server_processes > 1, offload to other processes.
sub_workers = await asyncio.gather(*(
_create_sub_worker() for _ in range(config.server_processes)))
calls = [worker.stub.RunServer() for worker in sub_workers]
config_request.setup.server_processes = 1
for call in calls:
await call.write(config_request)
# An empty status indicates the peer is ready
await call.read()
start_time = time.monotonic()
await context.write(
_get_server_status(
start_time,
start_time,
config.port,
))
_LOGGER.info('Servers are ready to serve.')
async for request in request_iterator:
end_time = time.monotonic()
for call in calls:
await call.write(request)
# Reports from sub workers doesn't matter
await call.read()
status = _get_server_status(
start_time,
end_time,
config.port,
)
if request.mark.reset:
start_time = end_time
await context.write(status)
for call in calls:
await call.done_writing()
for worker in sub_workers:
await worker.stub.QuitWorker(control_pb2.Void())
await worker.channel.close()
_LOGGER.info('Waiting for [%s] to quit...', worker)
await worker.process.wait()
async def _run_single_client(self, config, request_iterator, context):
running_tasks = []
qps_data = histogram.Histogram(config.histogram_params.resolution,
config.histogram_params.max_possible)
start_time = time.monotonic()
# Create a client for each channel as asyncio.Task
for i in range(config.client_channels):
server = config.server_targets[i % len(config.server_targets)]
client = _create_client(server, config, qps_data)
_LOGGER.info('Client created against server [%s]', server)
running_tasks.append(self._loop.create_task(client.run()))
end_time = time.monotonic()
await context.write(_get_client_status(start_time, end_time, qps_data))
# Respond to stat requests
async for request in request_iterator:
end_time = time.monotonic()
status = _get_client_status(start_time, end_time, qps_data)
if request.mark.reset:
qps_data.reset()
start_time = time.monotonic()
await context.write(status)
# Cleanup the clients
for task in running_tasks:
task.cancel()
async def RunClient(self, request_iterator, context):
config_request = await context.read()
config = config_request.setup
_LOGGER.info('Received ClientConfig: %s', config)
if config.client_processes <= 0:
_LOGGER.info('client_processes can\'t be [%d]',
config.client_processes)
_LOGGER.info('Using client_processes == [%d]', _NUM_CORES)
config.client_processes = _NUM_CORES
if config.client_processes == 1:
# If client_processes == 1, run the benchmark in this process.
await self._run_single_client(config, request_iterator, context)
else:
# If client_processes > 1, offload the work to other processes.
sub_workers = await asyncio.gather(*(
_create_sub_worker() for _ in range(config.client_processes)))
calls = [worker.stub.RunClient() for worker in sub_workers]
config_request.setup.client_processes = 1
for call in calls:
await call.write(config_request)
# An empty status indicates the peer is ready
await call.read()
start_time = time.monotonic()
result = histogram.Histogram(config.histogram_params.resolution,
config.histogram_params.max_possible)
end_time = time.monotonic()
await context.write(_get_client_status(start_time, end_time,
result))
async for request in request_iterator:
end_time = time.monotonic()
for call in calls:
_LOGGER.debug('Fetching status...')
await call.write(request)
sub_status = await call.read()
result.merge(sub_status.stats.latencies)
_LOGGER.debug('Update from sub worker count=[%d]',
sub_status.stats.latencies.count)
status = _get_client_status(start_time, end_time, result)
if request.mark.reset:
result.reset()
start_time = time.monotonic()
_LOGGER.debug('Reporting count=[%d]',
status.stats.latencies.count)
await context.write(status)
for call in calls:
await call.done_writing()
for worker in sub_workers:
await worker.stub.QuitWorker(control_pb2.Void())
await worker.channel.close()
_LOGGER.info('Waiting for sub worker [%s] to quit...', worker)
await worker.process.wait()
_LOGGER.info('Sub worker [%s] quit', worker)
@staticmethod
async def CoreCount(unused_request, unused_context):
return control_pb2.CoreResponse(cores=_NUM_CORES)
async def QuitWorker(self, unused_request, unused_context):
_LOGGER.info('QuitWorker command received.')
self._quit_event.set()
return control_pb2.Void()
async def wait_for_quit(self):
await self._quit_event.wait()

File diff suppressed because one or more lines are too long

File diff suppressed because one or more lines are too long

@ -23,7 +23,7 @@ source tools/internal_ci/helper_scripts/prepare_build_linux_perf_multilang_rc
# TODO(jtattermusch): add back "node" language once the scenarios are passing again
# See https://github.com/grpc/grpc/issues/20234
tools/run_tests/run_performance_tests.py \
-l c++ csharp ruby java python go php7 php7_protobuf_c \
-l c++ csharp ruby java python python_asyncio go php7 php7_protobuf_c \
--netperf \
--category smoketest \
-u kbuilder \

@ -21,7 +21,7 @@ source tools/internal_ci/helper_scripts/prepare_build_linux_perf_multilang_rc
# run 8core client vs 8core server
tools/run_tests/run_performance_tests.py \
-l c++ csharp ruby java python go php7 php7_protobuf_c node \
-l c++ csharp ruby java python python_asyncio go php7 php7_protobuf_c node \
--netperf \
--category scalable \
--remote_worker_host grpc-kokoro-performance-server-8core grpc-kokoro-performance-client-8core grpc-kokoro-performance-client2-8core \

@ -21,7 +21,7 @@ source tools/internal_ci/helper_scripts/prepare_build_linux_perf_multilang_rc
# run 8core client vs 8core server
tools/run_tests/run_performance_tests.py \
-l c++ csharp ruby java python go php7 php7_protobuf_c \
-l c++ csharp ruby java python python_asyncio go php7 php7_protobuf_c \
--netperf \
--category scalable \
--remote_worker_host grpc-kokoro-performance-server-8core grpc-kokoro-performance-client-8core grpc-kokoro-performance-client2-8core \

File diff suppressed because it is too large Load Diff

@ -18,6 +18,7 @@ source ~/.rvm/scripts/rvm
set -ex
cd "$(dirname "$0")/../../.."
bazel=$(pwd)/tools/bazel
CONFIG=${CONFIG:-opt}
@ -72,6 +73,9 @@ do
# python workers are only run with python2.7 and building with multiple python versions is costly
python tools/run_tests/run_tests.py -l "$language" -c "$CONFIG" --compiler python2.7 --build_only -j 8
;;
"python_asyncio")
$bazel build -c opt //src/python/grpcio_tests/tests_aio/benchmark:worker
;;
*)
python tools/run_tests/run_tests.py -l "$language" -c "$CONFIG" --build_only -j 8
;;

@ -0,0 +1,20 @@
#!/bin/bash
# Copyright 2020 The gRPC Authors
#
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
# You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
set -ex
cd "$(dirname "$0")/../../.."
bazel-bin/src/python/grpcio_tests/tests_aio/benchmark/worker "$@"

@ -119,6 +119,8 @@ def _ping_pong_scenario(name,
client_language=None,
server_language=None,
async_server_threads=0,
client_processes=0,
server_processes=0,
server_threads_per_cq=0,
client_threads_per_cq=0,
warmup_seconds=WARMUP_SECONDS,
@ -142,6 +144,7 @@ def _ping_pong_scenario(name,
'outstanding_rpcs_per_channel': 1,
'client_channels': 1,
'async_client_threads': 1,
'client_processes': client_processes,
'threads_per_cq': client_threads_per_cq,
'rpc_type': rpc_type,
'histogram_params': HISTOGRAM_PARAMS,
@ -151,6 +154,7 @@ def _ping_pong_scenario(name,
'server_type': server_type,
'security_params': _get_secargs(secure),
'async_server_threads': async_server_threads,
'server_processes': server_processes,
'threads_per_cq': server_threads_per_cq,
'channel_args': [],
},
@ -219,7 +223,7 @@ def _ping_pong_scenario(name,
scenario['SERVER_LANGUAGE'] = server_language
if categories:
scenario['CATEGORIES'] = categories
if len(excluded_poll_engines):
if excluded_poll_engines:
# The polling engines for which this scenario is excluded
scenario['EXCLUDED_POLL_ENGINES'] = excluded_poll_engines
return scenario
@ -793,7 +797,7 @@ class PythonLanguage:
client_type='SYNC_CLIENT',
server_type='ASYNC_SERVER',
server_language='c++',
async_server_threads=1,
async_server_threads=0,
categories=[SMOKETEST, SCALABLE])
yield _ping_pong_scenario(
@ -816,6 +820,154 @@ class PythonLanguage:
return 'python'
class PythonAsyncIOLanguage:
def __init__(self):
self.safename = 'python_asyncio'
def worker_cmdline(self):
return ['tools/run_tests/performance/run_worker_python_asyncio.sh']
def worker_port_offset(self):
return 1200
def scenarios(self):
for outstanding in [64, 128, 256, 512]:
for channels in [1, 4]:
yield _ping_pong_scenario(
'python_asyncio_protobuf_async_unary_ping_pong_%dx%d_max' %
(
outstanding,
channels,
),
rpc_type='UNARY',
client_type='ASYNC_CLIENT',
server_type='ASYNC_SERVER',
outstanding=outstanding * channels,
channels=channels,
client_processes=0,
server_processes=0,
unconstrained_client='async',
categories=[SCALABLE])
yield _ping_pong_scenario(
'python_asyncio_protobuf_async_unary_ping_pong_%d_1thread' %
outstanding,
rpc_type='UNARY',
client_type='ASYNC_CLIENT',
server_type='ASYNC_SERVER',
outstanding=outstanding,
channels=1,
client_processes=1,
server_processes=1,
unconstrained_client='async',
categories=[SCALABLE])
yield _ping_pong_scenario(
'python_asyncio_generic_async_streaming_ping_pong',
rpc_type='STREAMING',
client_type='ASYNC_CLIENT',
server_type='ASYNC_GENERIC_SERVER',
channels=1,
client_processes=1,
server_processes=1,
use_generic_payload=True,
categories=[SMOKETEST, SCALABLE])
yield _ping_pong_scenario(
'python_asyncio_protobuf_async_streaming_ping_pong',
rpc_type='STREAMING',
client_type='ASYNC_CLIENT',
server_type='ASYNC_SERVER',
channels=1,
client_processes=1,
server_processes=1,
categories=[SMOKETEST, SCALABLE])
yield _ping_pong_scenario(
'python_asyncio_protobuf_async_unary_ping_pong',
rpc_type='UNARY',
client_type='ASYNC_CLIENT',
server_type='ASYNC_SERVER',
client_processes=1,
server_processes=1,
categories=[SMOKETEST, SCALABLE])
yield _ping_pong_scenario(
'python_asyncio_protobuf_async_unary_ping_pong',
rpc_type='UNARY',
client_type='ASYNC_CLIENT',
server_type='ASYNC_SERVER',
channels=1,
client_processes=1,
server_processes=1,
categories=[SMOKETEST, SCALABLE])
yield _ping_pong_scenario(
'python_asyncio_protobuf_async_unary_qps_unconstrained',
rpc_type='UNARY',
client_type='ASYNC_CLIENT',
server_type='ASYNC_SERVER',
channels=1,
unconstrained_client='async')
yield _ping_pong_scenario(
'python_asyncio_protobuf_async_streaming_qps_unconstrained',
rpc_type='STREAMING',
client_type='ASYNC_CLIENT',
server_type='ASYNC_SERVER',
channels=1,
unconstrained_client='async')
yield _ping_pong_scenario(
'python_asyncio_to_cpp_protobuf_async_unary_ping_pong_1thread',
rpc_type='UNARY',
client_type='ASYNC_CLIENT',
server_type='ASYNC_SERVER',
server_language='c++',
channels=1,
client_processes=1,
unconstrained_client='async',
categories=[SMOKETEST, SCALABLE])
yield _ping_pong_scenario(
'python_asyncio_to_cpp_protobuf_async_unary_ping_pong_max',
rpc_type='UNARY',
client_type='ASYNC_CLIENT',
server_type='ASYNC_SERVER',
unconstrained_client='async',
channels=1,
client_processes=0,
server_language='c++',
categories=[SMOKETEST, SCALABLE])
yield _ping_pong_scenario(
'python_asyncio_to_cpp_protobuf_sync_streaming_ping_pong_1thread',
rpc_type='STREAMING',
client_type='ASYNC_CLIENT',
server_type='ASYNC_SERVER',
channels=1,
client_processes=1,
server_processes=1,
unconstrained_client='async',
server_language='c++')
yield _ping_pong_scenario(
'python_asyncio_protobuf_async_unary_ping_pong_1MB',
rpc_type='UNARY',
client_type='ASYNC_CLIENT',
server_type='ASYNC_SERVER',
req_size=1024 * 1024,
resp_size=1024 * 1024,
channels=1,
client_processes=1,
server_processes=1,
categories=[SMOKETEST, SCALABLE])
def __str__(self):
return 'python_asyncio'
class RubyLanguage:
def __init__(self):
@ -1249,6 +1401,7 @@ LANGUAGES = {
'php7_protobuf_c': Php7Language(php7_protobuf_c=True),
'java': JavaLanguage(),
'python': PythonLanguage(),
'python_asyncio': PythonAsyncIOLanguage(),
'go': GoLanguage(),
'node': NodeLanguage(),
'node_purejs': NodeLanguage(node_purejs=True)

Loading…
Cancel
Save