Merge branch 'qps_failya' into flow++

pull/12915/head
Craig Tiller 7 years ago
commit f90f69d1ca
  1. 4
      src/core/ext/transport/chttp2/transport/internal.h
  2. 686
      src/core/ext/transport/chttp2/transport/writing.cc
  3. 2
      src/core/lib/iomgr/ev_epollsig_linux.cc
  4. 56
      src/php/tests/qps/client.php
  5. 33
      src/php/tests/qps/generated_code/GPBMetadata/Src/Proto/Grpc/Core/Stats.php
  6. 215
      src/php/tests/qps/generated_code/GPBMetadata/Src/Proto/Grpc/Testing/Control.php
  7. 21
      src/php/tests/qps/generated_code/GPBMetadata/Src/Proto/Grpc/Testing/ProxyService.php
  8. 53
      src/php/tests/qps/generated_code/GPBMetadata/Src/Proto/Grpc/Testing/Services.php
  9. 47
      src/php/tests/qps/generated_code/GPBMetadata/Src/Proto/Grpc/Testing/Stats.php
  10. 75
      src/php/tests/qps/generated_code/Grpc/Core/Bucket.php
  11. 49
      src/php/tests/qps/generated_code/Grpc/Core/Histogram.php
  12. 102
      src/php/tests/qps/generated_code/Grpc/Core/Metric.php
  13. 49
      src/php/tests/qps/generated_code/Grpc/Core/Stats.php
  14. 68
      src/php/tests/qps/generated_code/Grpc/Testing/BenchmarkServiceClient.php
  15. 21
      src/php/tests/qps/generated_code/Grpc/Testing/BoolValue.php
  16. 24
      src/php/tests/qps/generated_code/Grpc/Testing/ByteBufferParams.php
  17. 34
      src/php/tests/qps/generated_code/Grpc/Testing/ChannelArg.php
  18. 27
      src/php/tests/qps/generated_code/Grpc/Testing/ClientArgs.php
  19. 288
      src/php/tests/qps/generated_code/Grpc/Testing/ClientConfig.php
  20. 147
      src/php/tests/qps/generated_code/Grpc/Testing/ClientStats.php
  21. 15
      src/php/tests/qps/generated_code/Grpc/Testing/ClientStatus.php
  22. 12
      src/php/tests/qps/generated_code/Grpc/Testing/ClientType.php
  23. 4
      src/php/tests/qps/generated_code/Grpc/Testing/ClosedLoopParams.php
  24. 4
      src/php/tests/qps/generated_code/Grpc/Testing/ComplexProtoParams.php
  25. 2
      src/php/tests/qps/generated_code/Grpc/Testing/CoreRequest.php
  26. 19
      src/php/tests/qps/generated_code/Grpc/Testing/CoreResponse.php
  27. 26
      src/php/tests/qps/generated_code/Grpc/Testing/EchoStatus.php
  28. 76
      src/php/tests/qps/generated_code/Grpc/Testing/HistogramData.php
  29. 38
      src/php/tests/qps/generated_code/Grpc/Testing/HistogramParams.php
  30. 27
      src/php/tests/qps/generated_code/Grpc/Testing/LoadParams.php
  31. 21
      src/php/tests/qps/generated_code/Grpc/Testing/Mark.php
  32. 38
      src/php/tests/qps/generated_code/Grpc/Testing/Payload.php
  33. 38
      src/php/tests/qps/generated_code/Grpc/Testing/PayloadConfig.php
  34. 8
      src/php/tests/qps/generated_code/Grpc/Testing/PayloadType.php
  35. 21
      src/php/tests/qps/generated_code/Grpc/Testing/PoissonParams.php
  36. 34
      src/php/tests/qps/generated_code/Grpc/Testing/ProxyClientServiceClient.php
  37. 13
      src/php/tests/qps/generated_code/Grpc/Testing/ProxyStat.php
  38. 32
      src/php/tests/qps/generated_code/Grpc/Testing/ReconnectInfo.php
  39. 15
      src/php/tests/qps/generated_code/Grpc/Testing/ReconnectParams.php
  40. 50
      src/php/tests/qps/generated_code/Grpc/Testing/ReportQpsScenarioServiceClient.php
  41. 24
      src/php/tests/qps/generated_code/Grpc/Testing/RequestResultCount.php
  42. 57
      src/php/tests/qps/generated_code/Grpc/Testing/ResponseParameters.php
  43. 18
      src/php/tests/qps/generated_code/Grpc/Testing/RpcType.php
  44. 144
      src/php/tests/qps/generated_code/Grpc/Testing/Scenario.php
  45. 193
      src/php/tests/qps/generated_code/Grpc/Testing/ScenarioResult.php
  46. 352
      src/php/tests/qps/generated_code/Grpc/Testing/ScenarioResultSummary.php
  47. 21
      src/php/tests/qps/generated_code/Grpc/Testing/Scenarios.php
  48. 52
      src/php/tests/qps/generated_code/Grpc/Testing/SecurityParams.php
  49. 27
      src/php/tests/qps/generated_code/Grpc/Testing/ServerArgs.php
  50. 211
      src/php/tests/qps/generated_code/Grpc/Testing/ServerConfig.php
  51. 151
      src/php/tests/qps/generated_code/Grpc/Testing/ServerStats.php
  52. 49
      src/php/tests/qps/generated_code/Grpc/Testing/ServerStatus.php
  53. 12
      src/php/tests/qps/generated_code/Grpc/Testing/ServerType.php
  54. 24
      src/php/tests/qps/generated_code/Grpc/Testing/SimpleProtoParams.php
  55. 148
      src/php/tests/qps/generated_code/Grpc/Testing/SimpleRequest.php
  56. 57
      src/php/tests/qps/generated_code/Grpc/Testing/SimpleResponse.php
  57. 42
      src/php/tests/qps/generated_code/Grpc/Testing/StreamingInputCallRequest.php
  58. 21
      src/php/tests/qps/generated_code/Grpc/Testing/StreamingInputCallResponse.php
  59. 82
      src/php/tests/qps/generated_code/Grpc/Testing/StreamingOutputCallRequest.php
  60. 23
      src/php/tests/qps/generated_code/Grpc/Testing/StreamingOutputCallResponse.php
  61. 2
      src/php/tests/qps/generated_code/Grpc/Testing/Void.php
  62. 40
      src/php/tests/qps/generated_code/Grpc/Testing/WorkerServiceClient.php
  63. 93
      src/php/tests/qps/histogram.php
  64. 2
      src/proto/grpc/testing/proxy-service.proto
  65. 15
      src/ruby/qps/histogram.rb
  66. 50
      src/ruby/qps/proxy-worker.rb
  67. 1
      src/ruby/qps/src/proto/grpc/testing/proxy-service_pb.rb
  68. 1
      src/ruby/qps/src/proto/grpc/testing/proxy-service_services_pb.rb
  69. 28
      test/cpp/qps/client.h
  70. 1
      test/cpp/qps/client_async.cc
  71. 56
      test/cpp/qps/client_sync.cc
  72. 11
      tools/internal_ci/helper_scripts/prepare_build_macos_rc
  73. 2
      tools/jenkins/run_full_performance.sh
  74. 9
      tools/run_tests/performance/build_performance.sh
  75. 30
      tools/run_tests/performance/build_performance_php7.sh
  76. 12
      tools/run_tests/performance/run_worker_php.sh
  77. 43
      tools/run_tests/performance/scenario_config.py
  78. 12
      tools/run_tests/python_utils/jobset.py

@ -607,8 +607,8 @@ bool grpc_chttp2_list_add_writable_stream(grpc_chttp2_transport *t,
returns non-zero if there was a stream available */
bool grpc_chttp2_list_pop_writable_stream(grpc_chttp2_transport *t,
grpc_chttp2_stream **s);
bool grpc_chttp2_list_remove_writable_stream(
grpc_chttp2_transport *t, grpc_chttp2_stream *s) GRPC_MUST_USE_RESULT;
bool grpc_chttp2_list_remove_writable_stream(grpc_chttp2_transport *t,
grpc_chttp2_stream *s);
bool grpc_chttp2_list_add_writing_stream(grpc_chttp2_transport *t,
grpc_chttp2_stream *s);

@ -174,343 +174,451 @@ static bool is_default_initial_metadata(grpc_metadata_batch *initial_metadata) {
return initial_metadata->list.default_count == initial_metadata->list.count;
}
grpc_chttp2_begin_write_result grpc_chttp2_begin_write(
grpc_exec_ctx *exec_ctx, grpc_chttp2_transport *t) {
grpc_chttp2_stream *s;
/* stats histogram counters: we increment these throughout this function,
and at the end publish to the central stats histograms */
int flow_control_writes = 0;
int initial_metadata_writes = 0;
int trailing_metadata_writes = 0;
int message_writes = 0;
namespace {
class StreamWriteContext;
class WriteContext {
public:
WriteContext(grpc_exec_ctx *exec_ctx, grpc_chttp2_transport *t) : t_(t) {
GRPC_STATS_INC_HTTP2_WRITES_BEGUN(exec_ctx);
GPR_TIMER_BEGIN("grpc_chttp2_begin_write", 0);
}
GRPC_STATS_INC_HTTP2_WRITES_BEGUN(exec_ctx);
// TODO(ctiller): make this the destructor
void FlushStats(grpc_exec_ctx *exec_ctx) {
GRPC_STATS_INC_HTTP2_SEND_INITIAL_METADATA_PER_WRITE(
exec_ctx, initial_metadata_writes_);
GRPC_STATS_INC_HTTP2_SEND_MESSAGE_PER_WRITE(exec_ctx, message_writes_);
GRPC_STATS_INC_HTTP2_SEND_TRAILING_METADATA_PER_WRITE(
exec_ctx, trailing_metadata_writes_);
GRPC_STATS_INC_HTTP2_SEND_FLOWCTL_PER_WRITE(exec_ctx, flow_control_writes_);
}
GPR_TIMER_BEGIN("grpc_chttp2_begin_write", 0);
void FlushSettings(grpc_exec_ctx *exec_ctx) {
if (t_->dirtied_local_settings && !t_->sent_local_settings) {
grpc_slice_buffer_add(
&t_->outbuf, grpc_chttp2_settings_create(
t_->settings[GRPC_SENT_SETTINGS],
t_->settings[GRPC_LOCAL_SETTINGS],
t_->force_send_settings, GRPC_CHTTP2_NUM_SETTINGS));
t_->force_send_settings = false;
t_->dirtied_local_settings = false;
t_->sent_local_settings = true;
GRPC_STATS_INC_HTTP2_SETTINGS_WRITES(exec_ctx);
}
}
if (t->dirtied_local_settings && !t->sent_local_settings) {
grpc_slice_buffer_add(
&t->outbuf,
grpc_chttp2_settings_create(
t->settings[GRPC_SENT_SETTINGS], t->settings[GRPC_LOCAL_SETTINGS],
t->force_send_settings, GRPC_CHTTP2_NUM_SETTINGS));
t->force_send_settings = 0;
t->dirtied_local_settings = 0;
t->sent_local_settings = 1;
GRPC_STATS_INC_HTTP2_SETTINGS_WRITES(exec_ctx);
void FlushQueuedBuffers(grpc_exec_ctx *exec_ctx) {
/* simple writes are queued to qbuf, and flushed here */
grpc_slice_buffer_move_into(&t_->qbuf, &t_->outbuf);
GPR_ASSERT(t_->qbuf.count == 0);
}
for (size_t i = 0; i < t->ping_ack_count; i++) {
grpc_slice_buffer_add(&t->outbuf,
grpc_chttp2_ping_create(1, t->ping_acks[i]));
void FlushWindowUpdates(grpc_exec_ctx *exec_ctx) {
uint32_t transport_announce =
grpc_chttp2_flowctl_maybe_send_transport_update(&t_->flow_control,
t_->outbuf.count > 0);
if (transport_announce) {
grpc_transport_one_way_stats throwaway_stats;
grpc_slice_buffer_add(
&t_->outbuf, grpc_chttp2_window_update_create(0, transport_announce,
&throwaway_stats));
ResetPingRecvClock();
}
}
t->ping_ack_count = 0;
/* simple writes are queued to qbuf, and flushed here */
grpc_slice_buffer_move_into(&t->qbuf, &t->outbuf);
GPR_ASSERT(t->qbuf.count == 0);
void FlushPingAcks() {
for (size_t i = 0; i < t_->ping_ack_count; i++) {
grpc_slice_buffer_add(&t_->outbuf,
grpc_chttp2_ping_create(true, t_->ping_acks[i]));
}
t_->ping_ack_count = 0;
}
grpc_chttp2_hpack_compressor_set_max_table_size(
&t->hpack_compressor,
t->settings[GRPC_PEER_SETTINGS][GRPC_CHTTP2_SETTINGS_HEADER_TABLE_SIZE]);
void EnactHpackSettings(grpc_exec_ctx *exec_ctx) {
grpc_chttp2_hpack_compressor_set_max_table_size(
&t_->hpack_compressor,
t_->settings[GRPC_PEER_SETTINGS]
[GRPC_CHTTP2_SETTINGS_HEADER_TABLE_SIZE]);
}
if (t->flow_control.remote_window > 0) {
while (grpc_chttp2_list_pop_stalled_by_transport(t, &s)) {
if (!t->closed && grpc_chttp2_list_add_writable_stream(t, s)) {
stream_ref_if_not_destroyed(&s->refcount->refs);
void UpdateStreamsNoLongerStalled() {
grpc_chttp2_stream *s;
while (grpc_chttp2_list_pop_stalled_by_transport(t_, &s)) {
if (!t_->closed && grpc_chttp2_list_add_writable_stream(t_, s)) {
if (!stream_ref_if_not_destroyed(&s->refcount->refs)) {
grpc_chttp2_list_remove_writable_stream(t_, s);
}
}
}
}
grpc_chttp2_begin_write_result result = {false, false, false};
grpc_chttp2_stream *NextStream() {
if (t_->outbuf.length > target_write_size(t_)) {
result_.partial = true;
return nullptr;
}
/* for each grpc_chttp2_stream that's become writable, frame it's data
(according to available window sizes) and add to the output buffer */
while (true) {
if (t->outbuf.length > target_write_size(t)) {
result.partial = true;
break;
grpc_chttp2_stream *s;
if (!grpc_chttp2_list_pop_writable_stream(t_, &s)) {
return nullptr;
}
return s;
}
void ResetPingRecvClock() {
if (!t_->is_client) {
t_->ping_recv_state.last_ping_recv_time = GRPC_MILLIS_INF_PAST;
t_->ping_recv_state.ping_strikes = 0;
}
}
void IncInitialMetadataWrites() { ++initial_metadata_writes_; }
void IncWindowUpdateWrites() { ++flow_control_writes_; }
void IncMessageWrites() { ++message_writes_; }
void IncTrailingMetadataWrites() { ++trailing_metadata_writes_; }
void NoteScheduledResults() { result_.early_results_scheduled = true; }
grpc_chttp2_transport *transport() const { return t_; }
grpc_chttp2_begin_write_result Result() {
result_.writing = t_->outbuf.count > 0;
return result_;
}
private:
grpc_chttp2_transport *const t_;
/* stats histogram counters: we increment these throughout this function,
and at the end publish to the central stats histograms */
int flow_control_writes_ = 0;
int initial_metadata_writes_ = 0;
int trailing_metadata_writes_ = 0;
int message_writes_ = 0;
grpc_chttp2_begin_write_result result_ = {false, false, false};
};
class DataSendContext {
public:
DataSendContext(WriteContext *write_context, grpc_chttp2_transport *t,
grpc_chttp2_stream *s)
: write_context_(write_context),
t_(t),
s_(s),
sending_bytes_before_(s_->sending_bytes) {}
uint32_t stream_remote_window() const {
return (uint32_t)GPR_MAX(
0, s_->flow_control.remote_window_delta +
(int64_t)t_->settings[GRPC_PEER_SETTINGS]
[GRPC_CHTTP2_SETTINGS_INITIAL_WINDOW_SIZE]);
}
uint32_t max_outgoing() const {
return (uint32_t)GPR_MIN(
t_->settings[GRPC_PEER_SETTINGS][GRPC_CHTTP2_SETTINGS_MAX_FRAME_SIZE],
GPR_MIN(stream_remote_window(), t_->flow_control.remote_window));
}
bool AnyOutgoing() const { return max_outgoing() != 0; }
void FlushCompressedBytes() {
uint32_t send_bytes =
(uint32_t)GPR_MIN(max_outgoing(), s_->compressed_data_buffer.length);
bool is_last_data_frame =
(send_bytes == s_->compressed_data_buffer.length &&
s_->flow_controlled_buffer.length == 0 &&
s_->fetching_send_message == NULL);
if (is_last_data_frame && s_->send_trailing_metadata != NULL &&
s_->stream_compression_ctx != NULL) {
if (!grpc_stream_compress(s_->stream_compression_ctx,
&s_->flow_controlled_buffer,
&s_->compressed_data_buffer, NULL, MAX_SIZE_T,
GRPC_STREAM_COMPRESSION_FLUSH_FINISH)) {
gpr_log(GPR_ERROR, "Stream compression failed.");
}
grpc_stream_compression_context_destroy(s_->stream_compression_ctx);
s_->stream_compression_ctx = NULL;
/* After finish, bytes in s->compressed_data_buffer may be
* more than max_outgoing. Start another round of the current
* while loop so that send_bytes and is_last_data_frame are
* recalculated. */
return;
}
is_last_frame_ = is_last_data_frame && s_->send_trailing_metadata != NULL &&
grpc_metadata_batch_is_empty(s_->send_trailing_metadata);
grpc_chttp2_encode_data(s_->id, &s_->compressed_data_buffer, send_bytes,
is_last_frame_, &s_->stats.outgoing, &t_->outbuf);
grpc_chttp2_flowctl_sent_data(&t_->flow_control, &s_->flow_control,
send_bytes);
if (s_->compressed_data_buffer.length == 0) {
s_->sending_bytes += s_->uncompressed_data_size;
}
}
if (!grpc_chttp2_list_pop_writable_stream(t, &s)) {
break;
void CompressMoreBytes() {
if (s_->stream_compression_ctx == NULL) {
s_->stream_compression_ctx =
grpc_stream_compression_context_create(s_->stream_compression_method);
}
s_->uncompressed_data_size = s_->flow_controlled_buffer.length;
if (!grpc_stream_compress(s_->stream_compression_ctx,
&s_->flow_controlled_buffer,
&s_->compressed_data_buffer, NULL, MAX_SIZE_T,
GRPC_STREAM_COMPRESSION_FLUSH_SYNC)) {
gpr_log(GPR_ERROR, "Stream compression failed.");
}
}
bool is_last_frame() const { return is_last_frame_; }
bool sent_initial_metadata = s->sent_initial_metadata;
bool now_writing = false;
void CallCallbacks(grpc_exec_ctx *exec_ctx) {
if (update_list(exec_ctx, t_, s_,
(int64_t)(s_->sending_bytes - sending_bytes_before_),
&s_->on_flow_controlled_cbs,
&s_->flow_controlled_bytes_flowed, GRPC_ERROR_NONE)) {
write_context_->NoteScheduledResults();
}
}
private:
WriteContext *write_context_;
grpc_chttp2_transport *t_;
grpc_chttp2_stream *s_;
const size_t sending_bytes_before_;
bool is_last_frame_ = false;
};
class StreamWriteContext {
public:
StreamWriteContext(WriteContext *write_context, grpc_chttp2_stream *s)
: write_context_(write_context), t_(write_context->transport()), s_(s) {
GRPC_CHTTP2_IF_TRACING(
gpr_log(GPR_DEBUG, "W:%p %s[%d] im-(sent,send)=(%d,%d) announce=%d", t,
t->is_client ? "CLIENT" : "SERVER", s->id,
sent_initial_metadata, s->send_initial_metadata != NULL,
gpr_log(GPR_DEBUG, "W:%p %s[%d] im-(sent,send)=(%d,%d) announce=%d", t_,
t_->is_client ? "CLIENT" : "SERVER", s->id,
s->sent_initial_metadata, s->send_initial_metadata != NULL,
(int)(s->flow_control.local_window_delta -
s->flow_control.announced_window_delta)));
}
grpc_mdelem *extra_headers_for_trailing_metadata[2];
size_t num_extra_headers_for_trailing_metadata = 0;
void FlushInitialMetadata(grpc_exec_ctx *exec_ctx) {
/* send initial metadata if it's available */
if (!sent_initial_metadata && s->send_initial_metadata != NULL) {
// We skip this on the server side if there is no custom initial
// metadata, there are no messages to send, and we are also sending
// trailing metadata. This results in a Trailers-Only response,
// which is required for retries, as per:
// https://github.com/grpc/proposal/blob/master/A6-client-retries.md#when-retries-are-valid
if (t->is_client || s->fetching_send_message != NULL ||
s->flow_controlled_buffer.length != 0 ||
s->send_trailing_metadata == NULL ||
!is_default_initial_metadata(s->send_initial_metadata)) {
grpc_encode_header_options hopt = {
s->id, // stream_id
false, // is_eof
t->settings[GRPC_PEER_SETTINGS]
[GRPC_CHTTP2_SETTINGS_GRPC_ALLOW_TRUE_BINARY_METADATA] !=
0, // use_true_binary_metadata
t->settings[GRPC_PEER_SETTINGS]
[GRPC_CHTTP2_SETTINGS_MAX_FRAME_SIZE], // max_frame_size
&s->stats.outgoing // stats
};
grpc_chttp2_encode_header(exec_ctx, &t->hpack_compressor, NULL, 0,
s->send_initial_metadata, &hopt, &t->outbuf);
now_writing = true;
if (!t->is_client) {
t->ping_recv_state.last_ping_recv_time = GRPC_MILLIS_INF_PAST;
t->ping_recv_state.ping_strikes = 0;
}
initial_metadata_writes++;
} else {
GRPC_CHTTP2_IF_TRACING(
gpr_log(GPR_INFO, "not sending initial_metadata (Trailers-Only)"));
// When sending Trailers-Only, we need to move the :status and
// content-type headers to the trailers.
if (s->send_initial_metadata->idx.named.status != NULL) {
extra_headers_for_trailing_metadata
[num_extra_headers_for_trailing_metadata++] =
&s->send_initial_metadata->idx.named.status->md;
}
if (s->send_initial_metadata->idx.named.content_type != NULL) {
extra_headers_for_trailing_metadata
[num_extra_headers_for_trailing_metadata++] =
&s->send_initial_metadata->idx.named.content_type->md;
}
trailing_metadata_writes++;
}
s->send_initial_metadata = NULL;
s->sent_initial_metadata = true;
sent_initial_metadata = true;
result.early_results_scheduled = true;
grpc_chttp2_complete_closure_step(
exec_ctx, t, s, &s->send_initial_metadata_finished, GRPC_ERROR_NONE,
"send_initial_metadata_finished");
if (s_->sent_initial_metadata) return;
if (s_->send_initial_metadata == nullptr) return;
// We skip this on the server side if there is no custom initial
// metadata, there are no messages to send, and we are also sending
// trailing metadata. This results in a Trailers-Only response,
// which is required for retries, as per:
// https://github.com/grpc/proposal/blob/master/A6-client-retries.md#when-retries-are-valid
if (!t_->is_client && s_->fetching_send_message == nullptr &&
s_->flow_controlled_buffer.length == 0 &&
s_->compressed_data_buffer.length == 0 &&
s_->send_trailing_metadata != nullptr &&
is_default_initial_metadata(s_->send_initial_metadata)) {
ConvertInitialMetadataToTrailingMetadata();
} else {
grpc_encode_header_options hopt = {
s_->id, // stream_id
false, // is_eof
t_->settings[GRPC_PEER_SETTINGS]
[GRPC_CHTTP2_SETTINGS_GRPC_ALLOW_TRUE_BINARY_METADATA] !=
0, // use_true_binary_metadata
t_->settings[GRPC_PEER_SETTINGS]
[GRPC_CHTTP2_SETTINGS_MAX_FRAME_SIZE], // max_frame_size
&s_->stats.outgoing // stats
};
grpc_chttp2_encode_header(exec_ctx, &t_->hpack_compressor, NULL, 0,
s_->send_initial_metadata, &hopt, &t_->outbuf);
write_context_->ResetPingRecvClock();
write_context_->IncInitialMetadataWrites();
}
s_->send_initial_metadata = NULL;
s_->sent_initial_metadata = true;
write_context_->NoteScheduledResults();
grpc_chttp2_complete_closure_step(
exec_ctx, t_, s_, &s_->send_initial_metadata_finished, GRPC_ERROR_NONE,
"send_initial_metadata_finished");
}
void FlushWindowUpdates(grpc_exec_ctx *exec_ctx) {
/* send any window updates */
uint32_t stream_announce = grpc_chttp2_flowctl_maybe_send_stream_update(
&t->flow_control, &s->flow_control);
if (stream_announce > 0) {
grpc_slice_buffer_add(
&t->outbuf, grpc_chttp2_window_update_create(s->id, stream_announce,
&s->stats.outgoing));
if (!t->is_client) {
t->ping_recv_state.last_ping_recv_time = GRPC_MILLIS_INF_PAST;
t->ping_recv_state.ping_strikes = 0;
&t_->flow_control, &s_->flow_control);
if (stream_announce == 0) return;
grpc_slice_buffer_add(
&t_->outbuf, grpc_chttp2_window_update_create(s_->id, stream_announce,
&s_->stats.outgoing));
write_context_->ResetPingRecvClock();
write_context_->IncWindowUpdateWrites();
}
void FlushData(grpc_exec_ctx *exec_ctx) {
if (!s_->sent_initial_metadata) return;
if (s_->flow_controlled_buffer.length == 0 &&
s_->compressed_data_buffer.length == 0) {
return; // early out: nothing to do
}
DataSendContext data_send_context(write_context_, t_, s_);
if (!data_send_context.AnyOutgoing()) {
if (t_->flow_control.remote_window == 0) {
report_stall(t_, s_, "transport");
grpc_chttp2_list_add_stalled_by_transport(t_, s_);
} else if (data_send_context.stream_remote_window() == 0) {
report_stall(t_, s_, "stream");
grpc_chttp2_list_add_stalled_by_stream(t_, s_);
}
flow_control_writes++;
return; // early out: nothing to do
}
if (sent_initial_metadata) {
/* send any body bytes, if allowed by flow control */
if (s->flow_controlled_buffer.length > 0 ||
s->compressed_data_buffer.length > 0) {
uint32_t stream_remote_window = (uint32_t)GPR_MAX(
0,
s->flow_control.remote_window_delta +
(int64_t)t->settings[GRPC_PEER_SETTINGS]
[GRPC_CHTTP2_SETTINGS_INITIAL_WINDOW_SIZE]);
uint32_t max_outgoing = (uint32_t)GPR_MIN(
t->settings[GRPC_PEER_SETTINGS]
[GRPC_CHTTP2_SETTINGS_MAX_FRAME_SIZE],
GPR_MIN(stream_remote_window, t->flow_control.remote_window));
if (max_outgoing > 0) {
bool is_last_data_frame = false;
bool is_last_frame = false;
size_t sending_bytes_before = s->sending_bytes;
while ((s->flow_controlled_buffer.length > 0 ||
s->compressed_data_buffer.length > 0) &&
max_outgoing > 0) {
if (s->compressed_data_buffer.length > 0) {
uint32_t send_bytes = (uint32_t)GPR_MIN(
max_outgoing, s->compressed_data_buffer.length);
is_last_data_frame =
(send_bytes == s->compressed_data_buffer.length &&
s->flow_controlled_buffer.length == 0 &&
s->fetching_send_message == NULL);
if (is_last_data_frame && s->send_trailing_metadata != NULL &&
s->stream_compression_ctx != NULL) {
if (!grpc_stream_compress(
s->stream_compression_ctx, &s->flow_controlled_buffer,
&s->compressed_data_buffer, NULL, MAX_SIZE_T,
GRPC_STREAM_COMPRESSION_FLUSH_FINISH)) {
gpr_log(GPR_ERROR, "Stream compression failed.");
}
grpc_stream_compression_context_destroy(
s->stream_compression_ctx);
s->stream_compression_ctx = NULL;
/* After finish, bytes in s->compressed_data_buffer may be
* more than max_outgoing. Start another round of the current
* while loop so that send_bytes and is_last_data_frame are
* recalculated. */
continue;
}
is_last_frame =
is_last_data_frame && s->send_trailing_metadata != NULL &&
grpc_metadata_batch_is_empty(s->send_trailing_metadata);
grpc_chttp2_encode_data(s->id, &s->compressed_data_buffer,
send_bytes, is_last_frame,
&s->stats.outgoing, &t->outbuf);
grpc_chttp2_flowctl_sent_data(&t->flow_control, &s->flow_control,
send_bytes);
max_outgoing -= send_bytes;
if (s->compressed_data_buffer.length == 0) {
s->sending_bytes += s->uncompressed_data_size;
}
} else {
if (s->stream_compression_ctx == NULL) {
s->stream_compression_ctx =
grpc_stream_compression_context_create(
s->stream_compression_method);
}
s->uncompressed_data_size = s->flow_controlled_buffer.length;
if (!grpc_stream_compress(
s->stream_compression_ctx, &s->flow_controlled_buffer,
&s->compressed_data_buffer, NULL, MAX_SIZE_T,
GRPC_STREAM_COMPRESSION_FLUSH_SYNC)) {
gpr_log(GPR_ERROR, "Stream compression failed.");
}
}
}
if (!t->is_client) {
t->ping_recv_state.last_ping_recv_time = 0;
t->ping_recv_state.ping_strikes = 0;
}
if (is_last_frame) {
s->send_trailing_metadata = NULL;
s->sent_trailing_metadata = true;
if (!t->is_client && !s->read_closed) {
grpc_slice_buffer_add(&t->outbuf, grpc_chttp2_rst_stream_create(
s->id, GRPC_HTTP2_NO_ERROR,
&s->stats.outgoing));
}
grpc_chttp2_mark_stream_closed(exec_ctx, t, s, !t->is_client, 1,
GRPC_ERROR_NONE);
}
result.early_results_scheduled |=
update_list(exec_ctx, t, s,
(int64_t)(s->sending_bytes - sending_bytes_before),
&s->on_flow_controlled_cbs,
&s->flow_controlled_bytes_flowed, GRPC_ERROR_NONE);
now_writing = true;
if (s->flow_controlled_buffer.length > 0 ||
s->compressed_data_buffer.length > 0) {
GRPC_CHTTP2_STREAM_REF(s, "chttp2_writing:fork");
grpc_chttp2_list_add_writable_stream(t, s);
}
message_writes++;
} else if (t->flow_control.remote_window == 0) {
report_stall(t, s, "transport");
grpc_chttp2_list_add_stalled_by_transport(t, s);
now_writing = true;
} else if (stream_remote_window == 0) {
report_stall(t, s, "stream");
grpc_chttp2_list_add_stalled_by_stream(t, s);
now_writing = true;
}
while ((s_->flow_controlled_buffer.length > 0 ||
s_->compressed_data_buffer.length > 0) &&
data_send_context.max_outgoing() > 0) {
if (s_->compressed_data_buffer.length > 0) {
data_send_context.FlushCompressedBytes();
} else {
data_send_context.CompressMoreBytes();
}
if (s->send_trailing_metadata != NULL &&
s->fetching_send_message == NULL &&
s->flow_controlled_buffer.length == 0 &&
s->compressed_data_buffer.length == 0) {
GRPC_CHTTP2_IF_TRACING(gpr_log(GPR_INFO, "sending trailing_metadata"));
if (grpc_metadata_batch_is_empty(s->send_trailing_metadata)) {
grpc_chttp2_encode_data(s->id, &s->flow_controlled_buffer, 0, true,
&s->stats.outgoing, &t->outbuf);
} else {
grpc_encode_header_options hopt = {
s->id, true,
t->settings
[GRPC_PEER_SETTINGS]
}
write_context_->ResetPingRecvClock();
if (data_send_context.is_last_frame()) {
SentLastFrame(exec_ctx);
}
data_send_context.CallCallbacks(exec_ctx);
stream_became_writable_ = true;
if (s_->flow_controlled_buffer.length > 0 ||
s_->compressed_data_buffer.length > 0) {
GRPC_CHTTP2_STREAM_REF(s_, "chttp2_writing:fork");
grpc_chttp2_list_add_writable_stream(t_, s_);
}
write_context_->IncMessageWrites();
}
void FlushTrailingMetadata(grpc_exec_ctx *exec_ctx) {
if (!s_->sent_initial_metadata) return;
if (s_->send_trailing_metadata == NULL) return;
if (s_->fetching_send_message != NULL) return;
if (s_->flow_controlled_buffer.length != 0) return;
if (s_->compressed_data_buffer.length != 0) return;
GRPC_CHTTP2_IF_TRACING(gpr_log(GPR_INFO, "sending trailing_metadata"));
if (grpc_metadata_batch_is_empty(s_->send_trailing_metadata)) {
grpc_chttp2_encode_data(s_->id, &s_->flow_controlled_buffer, 0, true,
&s_->stats.outgoing, &t_->outbuf);
} else {
grpc_encode_header_options hopt = {
s_->id, true,
t_->settings[GRPC_PEER_SETTINGS]
[GRPC_CHTTP2_SETTINGS_GRPC_ALLOW_TRUE_BINARY_METADATA] !=
0,
t->settings[GRPC_PEER_SETTINGS]
[GRPC_CHTTP2_SETTINGS_MAX_FRAME_SIZE],
&s->stats.outgoing};
grpc_chttp2_encode_header(exec_ctx, &t->hpack_compressor,
extra_headers_for_trailing_metadata,
num_extra_headers_for_trailing_metadata,
s->send_trailing_metadata, &hopt,
&t->outbuf);
trailing_metadata_writes++;
}
s->send_trailing_metadata = NULL;
s->sent_trailing_metadata = true;
if (!t->is_client) {
t->ping_recv_state.last_ping_recv_time = GRPC_MILLIS_INF_PAST;
t->ping_recv_state.ping_strikes = 0;
}
if (!t->is_client && !s->read_closed) {
grpc_slice_buffer_add(
&t->outbuf, grpc_chttp2_rst_stream_create(
s->id, GRPC_HTTP2_NO_ERROR, &s->stats.outgoing));
}
grpc_chttp2_mark_stream_closed(exec_ctx, t, s, !t->is_client, 1,
GRPC_ERROR_NONE);
now_writing = true;
result.early_results_scheduled = true;
grpc_chttp2_complete_closure_step(
exec_ctx, t, s, &s->send_trailing_metadata_finished,
GRPC_ERROR_NONE, "send_trailing_metadata_finished");
}
0,
t_->settings[GRPC_PEER_SETTINGS][GRPC_CHTTP2_SETTINGS_MAX_FRAME_SIZE],
&s_->stats.outgoing};
grpc_chttp2_encode_header(exec_ctx, &t_->hpack_compressor,
extra_headers_for_trailing_metadata_,
num_extra_headers_for_trailing_metadata_,
s_->send_trailing_metadata, &hopt, &t_->outbuf);
}
write_context_->IncTrailingMetadataWrites();
write_context_->ResetPingRecvClock();
SentLastFrame(exec_ctx);
write_context_->NoteScheduledResults();
grpc_chttp2_complete_closure_step(
exec_ctx, t_, s_, &s_->send_trailing_metadata_finished, GRPC_ERROR_NONE,
"send_trailing_metadata_finished");
}
bool stream_became_writable() { return stream_became_writable_; }
private:
void ConvertInitialMetadataToTrailingMetadata() {
GRPC_CHTTP2_IF_TRACING(
gpr_log(GPR_INFO, "not sending initial_metadata (Trailers-Only)"));
// When sending Trailers-Only, we need to move the :status and
// content-type headers to the trailers.
if (s_->send_initial_metadata->idx.named.status != NULL) {
extra_headers_for_trailing_metadata_
[num_extra_headers_for_trailing_metadata_++] =
&s_->send_initial_metadata->idx.named.status->md;
}
if (s_->send_initial_metadata->idx.named.content_type != NULL) {
extra_headers_for_trailing_metadata_
[num_extra_headers_for_trailing_metadata_++] =
&s_->send_initial_metadata->idx.named.content_type->md;
}
}
void SentLastFrame(grpc_exec_ctx *exec_ctx) {
s_->send_trailing_metadata = NULL;
s_->sent_trailing_metadata = true;
if (!t_->is_client && !s_->read_closed) {
grpc_slice_buffer_add(
&t_->outbuf, grpc_chttp2_rst_stream_create(
s_->id, GRPC_HTTP2_NO_ERROR, &s_->stats.outgoing));
}
grpc_chttp2_mark_stream_closed(exec_ctx, t_, s_, !t_->is_client, true,
GRPC_ERROR_NONE);
}
if (now_writing) {
GRPC_STATS_INC_HTTP2_SEND_INITIAL_METADATA_PER_WRITE(
exec_ctx, initial_metadata_writes);
GRPC_STATS_INC_HTTP2_SEND_MESSAGE_PER_WRITE(exec_ctx, message_writes);
GRPC_STATS_INC_HTTP2_SEND_TRAILING_METADATA_PER_WRITE(
exec_ctx, trailing_metadata_writes);
GRPC_STATS_INC_HTTP2_SEND_FLOWCTL_PER_WRITE(exec_ctx,
flow_control_writes);
WriteContext *const write_context_;
grpc_chttp2_transport *const t_;
grpc_chttp2_stream *const s_;
bool stream_became_writable_ = false;
grpc_mdelem *extra_headers_for_trailing_metadata_[2];
size_t num_extra_headers_for_trailing_metadata_ = 0;
};
} // namespace
grpc_chttp2_begin_write_result grpc_chttp2_begin_write(
grpc_exec_ctx *exec_ctx, grpc_chttp2_transport *t) {
WriteContext ctx(exec_ctx, t);
ctx.FlushSettings(exec_ctx);
ctx.FlushPingAcks();
ctx.FlushQueuedBuffers(exec_ctx);
ctx.EnactHpackSettings(exec_ctx);
if (t->flow_control.remote_window > 0) {
ctx.UpdateStreamsNoLongerStalled();
}
/* for each grpc_chttp2_stream that's become writable, frame it's data
(according to available window sizes) and add to the output buffer */
while (grpc_chttp2_stream *s = ctx.NextStream()) {
StreamWriteContext stream_ctx(&ctx, s);
stream_ctx.FlushInitialMetadata(exec_ctx);
stream_ctx.FlushWindowUpdates(exec_ctx);
stream_ctx.FlushData(exec_ctx);
stream_ctx.FlushTrailingMetadata(exec_ctx);
if (stream_ctx.stream_became_writable()) {
if (!grpc_chttp2_list_add_writing_stream(t, s)) {
/* already in writing list: drop ref */
GRPC_CHTTP2_STREAM_UNREF(exec_ctx, s, "chttp2_writing:already_writing");
} else {
/* ref will be dropped at end of write */
}
} else {
GRPC_CHTTP2_STREAM_UNREF(exec_ctx, s, "chttp2_writing:no_write");
}
}
maybe_initiate_ping(exec_ctx, t);
ctx.FlushWindowUpdates(exec_ctx);
uint32_t transport_announce = grpc_chttp2_flowctl_maybe_send_transport_update(
&t->flow_control, t->outbuf.count > 0);
if (transport_announce) {
grpc_transport_one_way_stats throwaway_stats;
grpc_slice_buffer_add(
&t->outbuf, grpc_chttp2_window_update_create(0, transport_announce,
&throwaway_stats));
if (!t->is_client) {
t->ping_recv_state.last_ping_recv_time = GRPC_MILLIS_INF_PAST;
t->ping_recv_state.ping_strikes = 0;
}
}
maybe_initiate_ping(exec_ctx, t);
GPR_TIMER_END("grpc_chttp2_begin_write", 0);
result.writing = t->outbuf.count > 0;
return result;
return ctx.Result();
}
void grpc_chttp2_end_write(grpc_exec_ctx *exec_ctx, grpc_chttp2_transport *t,

@ -18,6 +18,8 @@
#include "src/core/lib/iomgr/port.h"
#include <grpc/grpc_posix.h>
/* This polling engine is only relevant on linux kernels supporting epoll() */
#ifdef GRPC_LINUX_EPOLL

@ -37,6 +37,7 @@
*/
require dirname(__FILE__).'/vendor/autoload.php';
require dirname(__FILE__).'/histogram.php';
/**
* Assertion function that always exits with an error code if the assertion is
@ -63,19 +64,19 @@ function hardAssertIfStatusOk($status)
}
/* Start the actual client */
function qps_client_main($proxy_address) {
echo "Initiating php client\n";
function qps_client_main($proxy_address, $server_ind) {
echo "[php-client] Initiating php client\n";
$proxystubopts = [];
$proxystubopts['credentials'] = Grpc\ChannelCredentials::createInsecure();
$proxystub = new Grpc\Testing\ProxyClientServiceClient($proxy_address, $proxystubopts);
list($config, $status) = $proxystub->GetConfig(new Grpc\Testing\Void())->wait();
hardAssertIfStatusOk($status);
hardAssert($config->getClientChannels() == 1, "Only 1 channel supported");
hardAssert($config->getOutstandingRpcsPerChannel() == 1, "Only 1 outstanding RPC supported");
echo "Got configuration from proxy, target is " . $config->getServerTargets()[0] . "\n";
echo "[php-client] Got configuration from proxy, target is '$server_ind'th server" . $config->getServerTargets()[$server_ind] . "\n";
$histres = $config->getHistogramParams()->getResolution();
$histmax = $config->getHistogramParams()->getMaxPossible();
$stubopts = [];
if ($config->getSecurityParams()) {
@ -93,10 +94,10 @@ function qps_client_main($proxy_address) {
} else {
$stubopts['credentials'] = Grpc\ChannelCredentials::createInsecure();
}
echo "Initiating php benchmarking client\n";
echo "[php-client] Initiating php benchmarking client\n";
$stub = new Grpc\Testing\BenchmarkServiceClient(
$config->getServerTargets()[0], $stubopts);
$config->getServerTargets()[$server_ind], $stubopts);
$req = new Grpc\Testing\SimpleRequest();
$req->setResponseType(Grpc\Testing\PayloadType::COMPRESSABLE);
@ -115,8 +116,11 @@ function qps_client_main($proxy_address) {
} else {
$poisson = false;
}
$metric = new Grpc\Testing\ProxyStat;
$telemetry = $proxystub->ReportTime();
$histogram = new Histogram($histres, $histmax);
$histogram->clean();
$count = 0;
$histogram_result = new Grpc\Testing\HistogramData;
$telehist = $proxystub->ReportHist();
if ($config->getRpcType() == Grpc\Testing\RpcType::UNARY) {
while (1) {
if ($poisson) {
@ -126,8 +130,20 @@ function qps_client_main($proxy_address) {
$startreq = microtime(true);
list($resp,$status) = $stub->UnaryCall($req)->wait();
hardAssertIfStatusOk($status);
$metric->setLatency(microtime(true)-$startreq);
$telemetry->write($metric);
$histogram->add((microtime(true)-$startreq)*1e9);
$count += 1;
if ($count == 2000) {
$contents = $histogram->contents();
$histogram_result->setBucket($contents);
$histogram_result->setMinSeen($histogram->minimum());
$histogram_result->setMaxSeen($histogram->maximum());
$histogram_result->setSum($histogram->sum());
$histogram_result->setSumOfSquares($histogram->sum_of_squares());
$histogram_result->setCount($histogram->count());
$telehist->write($histogram_result);
$histogram->clean();
$count = 0;
}
}
} else {
$stream = $stub->StreamingCall();
@ -139,8 +155,20 @@ function qps_client_main($proxy_address) {
$startreq = microtime(true);
$stream->write($req);
$resp = $stream->read();
$metric->setLatency(microtime(true)-$startreq);
$telemetry->write($metric);
$histogram->add((microtime(true)-$startreq)*1e9);
$count += 1;
if ($count == 2000) {
$contents = $histogram->contents();
$histogram_result->setBucket($contents);
$histogram_result->setMinSeen($histogram->minimum());
$histogram_result->setMaxSeen($histogram->maximum());
$histogram_result->setSum($histogram->sum());
$histogram_result->setSumOfSquares($histogram->sum_of_squares());
$histogram_result->setCount($histogram->count());
$telehist->write($histogram_result);
$histogram->clean();
$count = 0;
}
}
}
}
@ -148,4 +176,4 @@ function qps_client_main($proxy_address) {
ini_set('display_startup_errors', 1);
ini_set('display_errors', 1);
error_reporting(-1);
qps_client_main($argv[1]);
qps_client_main($argv[1], $argv[2]);

@ -0,0 +1,33 @@
<?php
# Generated by the protocol buffer compiler. DO NOT EDIT!
# source: src/proto/grpc/core/stats.proto
namespace GPBMetadata\Src\Proto\Grpc\Core;
class Stats
{
public static $is_initialized = false;
public static function initOnce() {
$pool = \Google\Protobuf\Internal\DescriptorPool::getGeneratedPool();
if (static::$is_initialized == true) {
return;
}
$pool->internalAddGeneratedFile(hex2bin(
"0a97020a1f7372632f70726f746f2f677270632f636f72652f7374617473" .
"2e70726f746f1209677270632e636f726522260a064275636b6574120d0a" .
"057374617274180120012801120d0a05636f756e74180220012804222f0a" .
"09486973746f6772616d12220a076275636b65747318012003280b32112e" .
"677270632e636f72652e4275636b6574225b0a064d6574726963120c0a04" .
"6e616d65180120012809120f0a05636f756e74180a20012804480012290a" .
"09686973746f6772616d180b2001280b32142e677270632e636f72652e48" .
"6973746f6772616d480042070a0576616c7565222b0a0553746174731222" .
"0a076d65747269637318012003280b32112e677270632e636f72652e4d65" .
"74726963620670726f746f33"
));
static::$is_initialized = true;
}
}

@ -17,108 +17,119 @@ class Control
\GPBMetadata\Src\Proto\Grpc\Testing\Payloads::initOnce();
\GPBMetadata\Src\Proto\Grpc\Testing\Stats::initOnce();
$pool->internalAddGeneratedFile(hex2bin(
"0add170a247372632f70726f746f2f677270632f74657374696e672f636f" .
"6e74726f6c2e70726f746f120c677270632e74657374696e671a25737263" .
"2f70726f746f2f677270632f74657374696e672f7061796c6f6164732e70" .
"726f746f1a227372632f70726f746f2f677270632f74657374696e672f73" .
"746174732e70726f746f22250a0d506f6973736f6e506172616d7312140a" .
"0c6f6666657265645f6c6f616418012001280122120a10436c6f7365644c" .
"6f6f70506172616d73227b0a0a4c6f6164506172616d7312350a0b636c6f" .
"7365645f6c6f6f7018012001280b321e2e677270632e74657374696e672e" .
"436c6f7365644c6f6f70506172616d734800122e0a07706f6973736f6e18" .
"022001280b321b2e677270632e74657374696e672e506f6973736f6e5061" .
"72616d73480042060a046c6f616422430a0e536563757269747950617261" .
"6d7312130a0b7573655f746573745f6361180120012808121c0a14736572" .
"7665725f686f73745f6f76657272696465180220012809224d0a0a436861" .
"6e6e656c417267120c0a046e616d6518012001280912130a097374725f76" .
"616c7565180220012809480012130a09696e745f76616c75651803200128" .
"05480042070a0576616c756522a0040a0c436c69656e74436f6e66696712" .
"160a0e7365727665725f74617267657473180120032809122d0a0b636c69" .
"656e745f7479706518022001280e32182e677270632e74657374696e672e" .
"436c69656e745479706512350a0f73656375726974795f706172616d7318" .
"032001280b321c2e677270632e74657374696e672e536563757269747950" .
"6172616d7312240a1c6f75747374616e64696e675f727063735f7065725f" .
"6368616e6e656c18042001280512170a0f636c69656e745f6368616e6e65" .
"6c73180520012805121c0a146173796e635f636c69656e745f7468726561" .
"647318072001280512270a087270635f7479706518082001280e32152e67" .
"7270632e74657374696e672e52706354797065122d0a0b6c6f61645f7061" .
"72616d73180a2001280b32182e677270632e74657374696e672e4c6f6164" .
"506172616d7312330a0e7061796c6f61645f636f6e666967180b2001280b" .
"321b2e677270632e74657374696e672e5061796c6f6164436f6e66696712" .
"370a10686973746f6772616d5f706172616d73180c2001280b321d2e6772" .
"70632e74657374696e672e486973746f6772616d506172616d7312110a09" .
"636f72655f6c697374180d2003280512120a0a636f72655f6c696d697418" .
"0e2001280512180a106f746865725f636c69656e745f617069180f200128" .
"09122e0a0c6368616e6e656c5f6172677318102003280b32182e67727063" .
"2e74657374696e672e4368616e6e656c41726722380a0c436c69656e7453" .
"746174757312280a05737461747318012001280b32192e677270632e7465" .
"7374696e672e436c69656e74537461747322150a044d61726b120d0a0572" .
"6573657418012001280822680a0a436c69656e7441726773122b0a057365" .
"74757018012001280b321a2e677270632e74657374696e672e436c69656e" .
"74436f6e666967480012220a046d61726b18022001280b32122e67727063" .
"2e74657374696e672e4d61726b480042090a076172677479706522b4020a" .
"0c536572766572436f6e666967122d0a0b7365727665725f747970651801" .
"2001280e32182e677270632e74657374696e672e53657276657254797065" .
"12350a0f73656375726974795f706172616d7318022001280b321c2e6772" .
"70632e74657374696e672e5365637572697479506172616d73120c0a0470" .
"6f7274180420012805121c0a146173796e635f7365727665725f74687265" .
"61647318072001280512120a0a636f72655f6c696d697418082001280512" .
"330a0e7061796c6f61645f636f6e66696718092001280b321b2e67727063" .
"2e74657374696e672e5061796c6f6164436f6e66696712110a09636f7265" .
"5f6c697374180a2003280512180a106f746865725f7365727665725f6170" .
"69180b20012809121c0a137265736f757263655f71756f74615f73697a65" .
"18e9072001280522680a0a53657276657241726773122b0a057365747570" .
"18012001280b321a2e677270632e74657374696e672e536572766572436f" .
"6e666967480012220a046d61726b18022001280b32122e677270632e7465" .
"7374696e672e4d61726b480042090a076172677479706522550a0c536572" .
"76657253746174757312280a05737461747318012001280b32192e677270" .
"632e74657374696e672e5365727665725374617473120c0a04706f727418" .
"0220012805120d0a05636f726573180320012805220d0a0b436f72655265" .
"7175657374221d0a0c436f7265526573706f6e7365120d0a05636f726573" .
"18012001280522060a04566f696422fd010a085363656e6172696f120c0a" .
"046e616d6518012001280912310a0d636c69656e745f636f6e6669671802" .
"2001280b321a2e677270632e74657374696e672e436c69656e74436f6e66" .
"696712130a0b6e756d5f636c69656e747318032001280512310a0d736572" .
"7665725f636f6e66696718042001280b321a2e677270632e74657374696e" .
"672e536572766572436f6e66696712130a0b6e756d5f7365727665727318" .
"052001280512160a0e7761726d75705f7365636f6e647318062001280512" .
"190a1162656e63686d61726b5f7365636f6e647318072001280512200a18" .
"737061776e5f6c6f63616c5f776f726b65725f636f756e74180820012805" .
"22360a095363656e6172696f7312290a097363656e6172696f7318012003" .
"280b32162e677270632e74657374696e672e5363656e6172696f22f8020a" .
"155363656e6172696f526573756c7453756d6d617279120b0a0371707318" .
"0120012801121b0a137170735f7065725f7365727665725f636f72651802" .
"20012801121a0a127365727665725f73797374656d5f74696d6518032001" .
"280112180a107365727665725f757365725f74696d65180420012801121a" .
"0a12636c69656e745f73797374656d5f74696d6518052001280112180a10" .
"636c69656e745f757365725f74696d6518062001280112120a0a6c617465" .
"6e63795f353018072001280112120a0a6c6174656e63795f393018082001" .
"280112120a0a6c6174656e63795f393518092001280112120a0a6c617465" .
"6e63795f3939180a2001280112130a0b6c6174656e63795f393939180b20" .
"01280112180a107365727665725f6370755f7573616765180c2001280112" .
"260a1e7375636365737366756c5f72657175657374735f7065725f736563" .
"6f6e64180d2001280112220a1a6661696c65645f72657175657374735f70" .
"65725f7365636f6e64180e200128012283030a0e5363656e6172696f5265" .
"73756c7412280a087363656e6172696f18012001280b32162e677270632e" .
"74657374696e672e5363656e6172696f122e0a096c6174656e6369657318" .
"022001280b321b2e677270632e74657374696e672e486973746f6772616d" .
"44617461122f0a0c636c69656e745f737461747318032003280b32192e67" .
"7270632e74657374696e672e436c69656e745374617473122f0a0c736572" .
"7665725f737461747318042003280b32192e677270632e74657374696e67" .
"2e536572766572537461747312140a0c7365727665725f636f7265731805" .
"2003280512340a0773756d6d61727918062001280b32232e677270632e74" .
"657374696e672e5363656e6172696f526573756c7453756d6d6172791216" .
"0a0e636c69656e745f7375636365737318072003280812160a0e73657276" .
"65725f7375636365737318082003280812390a0f726571756573745f7265" .
"73756c747318092003280b32202e677270632e74657374696e672e526571" .
"75657374526573756c74436f756e742a410a0a436c69656e745479706512" .
"0f0a0b53594e435f434c49454e54100012100a0c4153594e435f434c4945" .
"4e54100112100a0c4f544845525f434c49454e5410022a5b0a0a53657276" .
"657254797065120f0a0b53594e435f534552564552100012100a0c415359" .
"4e435f534552564552100112180a144153594e435f47454e455249435f53" .
"4552564552100212100a0c4f544845525f53455256455210032a230a0752" .
"70635479706512090a05554e4152591000120d0a0953545245414d494e47" .
"1001620670726f746f33"
"0aa21a0a247372632f70726f746f2f677270632f74657374696e672f636f" .
"6e74726f6c2e70726f746f120c677270632e74657374696e671a22737263" .
"2f70726f746f2f677270632f74657374696e672f73746174732e70726f74" .
"6f22250a0d506f6973736f6e506172616d7312140a0c6f6666657265645f" .
"6c6f616418012001280122120a10436c6f7365644c6f6f70506172616d73" .
"227b0a0a4c6f6164506172616d7312350a0b636c6f7365645f6c6f6f7018" .
"012001280b321e2e677270632e74657374696e672e436c6f7365644c6f6f" .
"70506172616d734800122e0a07706f6973736f6e18022001280b321b2e67" .
"7270632e74657374696e672e506f6973736f6e506172616d73480042060a" .
"046c6f616422560a0e5365637572697479506172616d7312130a0b757365" .
"5f746573745f6361180120012808121c0a147365727665725f686f73745f" .
"6f7665727269646518022001280912110a09637265645f74797065180320" .
"012809224d0a0a4368616e6e656c417267120c0a046e616d651801200128" .
"0912130a097374725f76616c7565180220012809480012130a09696e745f" .
"76616c7565180320012805480042070a0576616c756522d5040a0c436c69" .
"656e74436f6e66696712160a0e7365727665725f74617267657473180120" .
"032809122d0a0b636c69656e745f7479706518022001280e32182e677270" .
"632e74657374696e672e436c69656e745479706512350a0f736563757269" .
"74795f706172616d7318032001280b321c2e677270632e74657374696e67" .
"2e5365637572697479506172616d7312240a1c6f75747374616e64696e67" .
"5f727063735f7065725f6368616e6e656c18042001280512170a0f636c69" .
"656e745f6368616e6e656c73180520012805121c0a146173796e635f636c" .
"69656e745f7468726561647318072001280512270a087270635f74797065" .
"18082001280e32152e677270632e74657374696e672e5270635479706512" .
"2d0a0b6c6f61645f706172616d73180a2001280b32182e677270632e7465" .
"7374696e672e4c6f6164506172616d7312330a0e7061796c6f61645f636f" .
"6e666967180b2001280b321b2e677270632e74657374696e672e5061796c" .
"6f6164436f6e66696712370a10686973746f6772616d5f706172616d7318" .
"0c2001280b321d2e677270632e74657374696e672e486973746f6772616d" .
"506172616d7312110a09636f72655f6c697374180d2003280512120a0a63" .
"6f72655f6c696d6974180e2001280512180a106f746865725f636c69656e" .
"745f617069180f20012809122e0a0c6368616e6e656c5f61726773181020" .
"03280b32182e677270632e74657374696e672e4368616e6e656c41726712" .
"160a0e746872656164735f7065725f6371181120012805121b0a136d6573" .
"73616765735f7065725f73747265616d18122001280522380a0c436c6965" .
"6e7453746174757312280a05737461747318012001280b32192e67727063" .
"2e74657374696e672e436c69656e74537461747322150a044d61726b120d" .
"0a05726573657418012001280822680a0a436c69656e7441726773122b0a" .
"05736574757018012001280b321a2e677270632e74657374696e672e436c" .
"69656e74436f6e666967480012220a046d61726b18022001280b32122e67" .
"7270632e74657374696e672e4d61726b480042090a076172677479706522" .
"fd020a0c536572766572436f6e666967122d0a0b7365727665725f747970" .
"6518012001280e32182e677270632e74657374696e672e53657276657254" .
"79706512350a0f73656375726974795f706172616d7318022001280b321c" .
"2e677270632e74657374696e672e5365637572697479506172616d73120c" .
"0a04706f7274180420012805121c0a146173796e635f7365727665725f74" .
"68726561647318072001280512120a0a636f72655f6c696d697418082001" .
"280512330a0e7061796c6f61645f636f6e66696718092001280b321b2e67" .
"7270632e74657374696e672e5061796c6f6164436f6e66696712110a0963" .
"6f72655f6c697374180a2003280512180a106f746865725f736572766572" .
"5f617069180b2001280912160a0e746872656164735f7065725f6371180c" .
"20012805121c0a137265736f757263655f71756f74615f73697a6518e907" .
"20012805122f0a0c6368616e6e656c5f6172677318ea072003280b32182e" .
"677270632e74657374696e672e4368616e6e656c41726722680a0a536572" .
"76657241726773122b0a05736574757018012001280b321a2e677270632e" .
"74657374696e672e536572766572436f6e666967480012220a046d61726b" .
"18022001280b32122e677270632e74657374696e672e4d61726b48004209" .
"0a076172677479706522550a0c53657276657253746174757312280a0573" .
"7461747318012001280b32192e677270632e74657374696e672e53657276" .
"65725374617473120c0a04706f7274180220012805120d0a05636f726573" .
"180320012805220d0a0b436f726552657175657374221d0a0c436f726552" .
"6573706f6e7365120d0a05636f72657318012001280522060a04566f6964" .
"22fd010a085363656e6172696f120c0a046e616d6518012001280912310a" .
"0d636c69656e745f636f6e66696718022001280b321a2e677270632e7465" .
"7374696e672e436c69656e74436f6e66696712130a0b6e756d5f636c6965" .
"6e747318032001280512310a0d7365727665725f636f6e66696718042001" .
"280b321a2e677270632e74657374696e672e536572766572436f6e666967" .
"12130a0b6e756d5f7365727665727318052001280512160a0e7761726d75" .
"705f7365636f6e647318062001280512190a1162656e63686d61726b5f73" .
"65636f6e647318072001280512200a18737061776e5f6c6f63616c5f776f" .
"726b65725f636f756e7418082001280522360a095363656e6172696f7312" .
"290a097363656e6172696f7318012003280b32162e677270632e74657374" .
"696e672e5363656e6172696f2284040a155363656e6172696f526573756c" .
"7453756d6d617279120b0a03717073180120012801121b0a137170735f70" .
"65725f7365727665725f636f7265180220012801121a0a12736572766572" .
"5f73797374656d5f74696d6518032001280112180a107365727665725f75" .
"7365725f74696d65180420012801121a0a12636c69656e745f7379737465" .
"6d5f74696d6518052001280112180a10636c69656e745f757365725f7469" .
"6d6518062001280112120a0a6c6174656e63795f35301807200128011212" .
"0a0a6c6174656e63795f393018082001280112120a0a6c6174656e63795f" .
"393518092001280112120a0a6c6174656e63795f3939180a200128011213" .
"0a0b6c6174656e63795f393939180b2001280112180a107365727665725f" .
"6370755f7573616765180c2001280112260a1e7375636365737366756c5f" .
"72657175657374735f7065725f7365636f6e64180d2001280112220a1a66" .
"61696c65645f72657175657374735f7065725f7365636f6e64180e200128" .
"0112200a18636c69656e745f706f6c6c735f7065725f7265717565737418" .
"0f2001280112200a187365727665725f706f6c6c735f7065725f72657175" .
"65737418102001280112220a1a7365727665725f717565726965735f7065" .
"725f6370755f73656318112001280112220a1a636c69656e745f71756572" .
"6965735f7065725f6370755f7365631812200128012283030a0e5363656e" .
"6172696f526573756c7412280a087363656e6172696f18012001280b3216" .
"2e677270632e74657374696e672e5363656e6172696f122e0a096c617465" .
"6e6369657318022001280b321b2e677270632e74657374696e672e486973" .
"746f6772616d44617461122f0a0c636c69656e745f737461747318032003" .
"280b32192e677270632e74657374696e672e436c69656e74537461747312" .
"2f0a0c7365727665725f737461747318042003280b32192e677270632e74" .
"657374696e672e536572766572537461747312140a0c7365727665725f63" .
"6f72657318052003280512340a0773756d6d61727918062001280b32232e" .
"677270632e74657374696e672e5363656e6172696f526573756c7453756d" .
"6d61727912160a0e636c69656e745f737563636573731807200328081216" .
"0a0e7365727665725f7375636365737318082003280812390a0f72657175" .
"6573745f726573756c747318092003280b32202e677270632e7465737469" .
"6e672e52657175657374526573756c74436f756e742a410a0a436c69656e" .
"7454797065120f0a0b53594e435f434c49454e54100012100a0c4153594e" .
"435f434c49454e54100112100a0c4f544845525f434c49454e5410022a5b" .
"0a0a53657276657254797065120f0a0b53594e435f534552564552100012" .
"100a0c4153594e435f534552564552100112180a144153594e435f47454e" .
"455249435f534552564552100212100a0c4f544845525f53455256455210" .
"032a720a075270635479706512090a05554e4152591000120d0a09535452" .
"45414d494e47100112190a1553545245414d494e475f46524f4d5f434c49" .
"454e54100212190a1553545245414d494e475f46524f4d5f534552564552" .
"100312170a1353545245414d494e475f424f54485f574159531004620670" .
"726f746f33"
));
static::$is_initialized = true;

@ -15,17 +15,20 @@ class ProxyService
return;
}
\GPBMetadata\Src\Proto\Grpc\Testing\Control::initOnce();
\GPBMetadata\Src\Proto\Grpc\Testing\Stats::initOnce();
$pool->internalAddGeneratedFile(hex2bin(
"0a97020a2a7372632f70726f746f2f677270632f74657374696e672f7072" .
"0ad6020a2a7372632f70726f746f2f677270632f74657374696e672f7072" .
"6f78792d736572766963652e70726f746f120c677270632e74657374696e" .
"671a247372632f70726f746f2f677270632f74657374696e672f636f6e74" .
"726f6c2e70726f746f221c0a0950726f787953746174120f0a076c617465" .
"6e6379180120012801328e010a1250726f7879436c69656e745365727669" .
"6365123b0a09476574436f6e66696712122e677270632e74657374696e67" .
"2e566f69641a1a2e677270632e74657374696e672e436c69656e74436f6e" .
"666967123b0a0a5265706f727454696d6512172e677270632e7465737469" .
"6e672e50726f7879537461741a122e677270632e74657374696e672e566f" .
"69642801620670726f746f33"
"671a227372632f70726f746f2f677270632f74657374696e672f73746174" .
"732e70726f746f221c0a0950726f787953746174120f0a076c6174656e63" .
"7918012001280132cf010a1250726f7879436c69656e7453657276696365" .
"123b0a09476574436f6e66696712122e677270632e74657374696e672e56" .
"6f69641a1a2e677270632e74657374696e672e436c69656e74436f6e6669" .
"67123b0a0a5265706f727454696d6512172e677270632e74657374696e67" .
"2e50726f7879537461741a122e677270632e74657374696e672e566f6964" .
"2801123f0a0a5265706f727448697374121b2e677270632e74657374696e" .
"672e486973746f6772616d446174611a122e677270632e74657374696e67" .
"2e566f69642801620670726f746f33"
));
static::$is_initialized = true;

@ -16,27 +16,40 @@ class Services
}
\GPBMetadata\Src\Proto\Grpc\Testing\Messages::initOnce();
\GPBMetadata\Src\Proto\Grpc\Testing\Control::initOnce();
\GPBMetadata\Src\Proto\Grpc\Testing\Stats::initOnce();
$pool->internalAddGeneratedFile(hex2bin(
"0ad1040a257372632f70726f746f2f677270632f74657374696e672f7365" .
"7276696365732e70726f746f120c677270632e74657374696e671a257372" .
"632f70726f746f2f677270632f74657374696e672f6d657373616765732e" .
"70726f746f1a247372632f70726f746f2f677270632f74657374696e672f" .
"636f6e74726f6c2e70726f746f32aa010a1042656e63686d61726b536572" .
"7669636512460a09556e61727943616c6c121b2e677270632e7465737469" .
"6e672e53696d706c65526571756573741a1c2e677270632e74657374696e" .
"672e53696d706c65526573706f6e7365124e0a0d53747265616d696e6743" .
"616c6c121b2e677270632e74657374696e672e53696d706c655265717565" .
"73741a1c2e677270632e74657374696e672e53696d706c65526573706f6e" .
"7365280130013297020a0d576f726b65725365727669636512450a095275" .
"6e53657276657212182e677270632e74657374696e672e53657276657241" .
"7267731a1a2e677270632e74657374696e672e5365727665725374617475" .
"732801300112450a0952756e436c69656e7412182e677270632e74657374" .
"696e672e436c69656e74417267731a1a2e677270632e74657374696e672e" .
"436c69656e745374617475732801300112420a09436f7265436f756e7412" .
"192e677270632e74657374696e672e436f7265526571756573741a1a2e67" .
"7270632e74657374696e672e436f7265526573706f6e736512340a0a5175" .
"6974576f726b657212122e677270632e74657374696e672e566f69641a12" .
"2e677270632e74657374696e672e566f6964620670726f746f33"
"0aaa070a257372632f70726f746f2f677270632f74657374696e672f7365" .
"7276696365732e70726f746f120c677270632e74657374696e671a247372" .
"632f70726f746f2f677270632f74657374696e672f636f6e74726f6c2e70" .
"726f746f1a227372632f70726f746f2f677270632f74657374696e672f73" .
"746174732e70726f746f32a6030a1042656e63686d61726b536572766963" .
"6512460a09556e61727943616c6c121b2e677270632e74657374696e672e" .
"53696d706c65526571756573741a1c2e677270632e74657374696e672e53" .
"696d706c65526573706f6e7365124e0a0d53747265616d696e6743616c6c" .
"121b2e677270632e74657374696e672e53696d706c65526571756573741a" .
"1c2e677270632e74657374696e672e53696d706c65526573706f6e736528" .
"01300112520a1353747265616d696e6746726f6d436c69656e74121b2e67" .
"7270632e74657374696e672e53696d706c65526571756573741a1c2e6772" .
"70632e74657374696e672e53696d706c65526573706f6e7365280112520a" .
"1353747265616d696e6746726f6d536572766572121b2e677270632e7465" .
"7374696e672e53696d706c65526571756573741a1c2e677270632e746573" .
"74696e672e53696d706c65526573706f6e7365300112520a115374726561" .
"6d696e67426f746857617973121b2e677270632e74657374696e672e5369" .
"6d706c65526571756573741a1c2e677270632e74657374696e672e53696d" .
"706c65526573706f6e7365280130013297020a0d576f726b657253657276" .
"69636512450a0952756e53657276657212182e677270632e74657374696e" .
"672e536572766572417267731a1a2e677270632e74657374696e672e5365" .
"727665725374617475732801300112450a0952756e436c69656e7412182e" .
"677270632e74657374696e672e436c69656e74417267731a1a2e67727063" .
"2e74657374696e672e436c69656e745374617475732801300112420a0943" .
"6f7265436f756e7412192e677270632e74657374696e672e436f72655265" .
"71756573741a1a2e677270632e74657374696e672e436f7265526573706f" .
"6e736512340a0a51756974576f726b657212122e677270632e7465737469" .
"6e672e566f69641a122e677270632e74657374696e672e566f6964325e0a" .
"185265706f72745170735363656e6172696f5365727669636512420a0e52" .
"65706f72745363656e6172696f121c2e677270632e74657374696e672e53" .
"63656e6172696f526573756c741a122e677270632e74657374696e672e56" .
"6f6964620670726f746f33"
));
static::$is_initialized = true;

@ -14,28 +14,33 @@ class Stats
if (static::$is_initialized == true) {
return;
}
\GPBMetadata\Src\Proto\Grpc\Core\Stats::initOnce();
$pool->internalAddGeneratedFile(hex2bin(
"0adf040a227372632f70726f746f2f677270632f74657374696e672f7374" .
"6174732e70726f746f120c677270632e74657374696e67227a0a0b536572" .
"766572537461747312140a0c74696d655f656c6170736564180120012801" .
"12110a0974696d655f7573657218022001280112130a0b74696d655f7379" .
"7374656d18032001280112160a0e746f74616c5f6370755f74696d651804" .
"2001280412150a0d69646c655f6370755f74696d65180520012804223b0a" .
"0f486973746f6772616d506172616d7312120a0a7265736f6c7574696f6e" .
"18012001280112140a0c6d61785f706f737369626c651802200128012277" .
"0a0d486973746f6772616d44617461120e0a066275636b65741801200328" .
"0d12100a086d696e5f7365656e18022001280112100a086d61785f736565" .
"6e180320012801120b0a0373756d18042001280112160a0e73756d5f6f66" .
"5f73717561726573180520012801120d0a05636f756e7418062001280122" .
"380a1252657175657374526573756c74436f756e7412130a0b7374617475" .
"735f636f6465180120012805120d0a05636f756e7418022001280322b601" .
"0a0b436c69656e745374617473122e0a096c6174656e6369657318012001" .
"280b321b2e677270632e74657374696e672e486973746f6772616d446174" .
"6112140a0c74696d655f656c617073656418022001280112110a0974696d" .
"655f7573657218032001280112130a0b74696d655f73797374656d180420" .
"01280112390a0f726571756573745f726573756c747318052003280b3220" .
"2e677270632e74657374696e672e52657175657374526573756c74436f75" .
"6e74620670726f746f33"
"0ada050a227372632f70726f746f2f677270632f74657374696e672f7374" .
"6174732e70726f746f120c677270632e74657374696e6722b7010a0b5365" .
"72766572537461747312140a0c74696d655f656c61707365641801200128" .
"0112110a0974696d655f7573657218022001280112130a0b74696d655f73" .
"797374656d18032001280112160a0e746f74616c5f6370755f74696d6518" .
"042001280412150a0d69646c655f6370755f74696d651805200128041215" .
"0a0d63715f706f6c6c5f636f756e7418062001280412240a0a636f72655f" .
"737461747318072001280b32102e677270632e636f72652e537461747322" .
"3b0a0f486973746f6772616d506172616d7312120a0a7265736f6c757469" .
"6f6e18012001280112140a0c6d61785f706f737369626c65180220012801" .
"22770a0d486973746f6772616d44617461120e0a066275636b6574180120" .
"03280d12100a086d696e5f7365656e18022001280112100a086d61785f73" .
"65656e180320012801120b0a0373756d18042001280112160a0e73756d5f" .
"6f665f73717561726573180520012801120d0a05636f756e741806200128" .
"0122380a1252657175657374526573756c74436f756e7412130a0b737461" .
"7475735f636f6465180120012805120d0a05636f756e7418022001280322" .
"f3010a0b436c69656e745374617473122e0a096c6174656e636965731801" .
"2001280b321b2e677270632e74657374696e672e486973746f6772616d44" .
"61746112140a0c74696d655f656c617073656418022001280112110a0974" .
"696d655f7573657218032001280112130a0b74696d655f73797374656d18" .
"042001280112390a0f726571756573745f726573756c747318052003280b" .
"32202e677270632e74657374696e672e52657175657374526573756c7443" .
"6f756e7412150a0d63715f706f6c6c5f636f756e7418062001280412240a" .
"0a636f72655f737461747318072001280b32102e677270632e636f72652e" .
"5374617473620670726f746f33"
));
static::$is_initialized = true;

@ -0,0 +1,75 @@
<?php
# Generated by the protocol buffer compiler. DO NOT EDIT!
# source: src/proto/grpc/core/stats.proto
namespace Grpc\Core;
use Google\Protobuf\Internal\GPBType;
use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* Generated from protobuf message <code>grpc.core.Bucket</code>
*/
class Bucket extends \Google\Protobuf\Internal\Message
{
/**
* Generated from protobuf field <code>double start = 1;</code>
*/
private $start = 0.0;
/**
* Generated from protobuf field <code>uint64 count = 2;</code>
*/
private $count = 0;
public function __construct() {
\GPBMetadata\Src\Proto\Grpc\Core\Stats::initOnce();
parent::__construct();
}
/**
* Generated from protobuf field <code>double start = 1;</code>
* @return float
*/
public function getStart()
{
return $this->start;
}
/**
* Generated from protobuf field <code>double start = 1;</code>
* @param float $var
* @return $this
*/
public function setStart($var)
{
GPBUtil::checkDouble($var);
$this->start = $var;
return $this;
}
/**
* Generated from protobuf field <code>uint64 count = 2;</code>
* @return int|string
*/
public function getCount()
{
return $this->count;
}
/**
* Generated from protobuf field <code>uint64 count = 2;</code>
* @param int|string $var
* @return $this
*/
public function setCount($var)
{
GPBUtil::checkUint64($var);
$this->count = $var;
return $this;
}
}

@ -0,0 +1,49 @@
<?php
# Generated by the protocol buffer compiler. DO NOT EDIT!
# source: src/proto/grpc/core/stats.proto
namespace Grpc\Core;
use Google\Protobuf\Internal\GPBType;
use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* Generated from protobuf message <code>grpc.core.Histogram</code>
*/
class Histogram extends \Google\Protobuf\Internal\Message
{
/**
* Generated from protobuf field <code>repeated .grpc.core.Bucket buckets = 1;</code>
*/
private $buckets;
public function __construct() {
\GPBMetadata\Src\Proto\Grpc\Core\Stats::initOnce();
parent::__construct();
}
/**
* Generated from protobuf field <code>repeated .grpc.core.Bucket buckets = 1;</code>
* @return \Google\Protobuf\Internal\RepeatedField
*/
public function getBuckets()
{
return $this->buckets;
}
/**
* Generated from protobuf field <code>repeated .grpc.core.Bucket buckets = 1;</code>
* @param \Grpc\Core\Bucket[]|\Google\Protobuf\Internal\RepeatedField $var
* @return $this
*/
public function setBuckets($var)
{
$arr = GPBUtil::checkRepeatedField($var, \Google\Protobuf\Internal\GPBType::MESSAGE, \Grpc\Core\Bucket::class);
$this->buckets = $arr;
return $this;
}
}

@ -0,0 +1,102 @@
<?php
# Generated by the protocol buffer compiler. DO NOT EDIT!
# source: src/proto/grpc/core/stats.proto
namespace Grpc\Core;
use Google\Protobuf\Internal\GPBType;
use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* Generated from protobuf message <code>grpc.core.Metric</code>
*/
class Metric extends \Google\Protobuf\Internal\Message
{
/**
* Generated from protobuf field <code>string name = 1;</code>
*/
private $name = '';
protected $value;
public function __construct() {
\GPBMetadata\Src\Proto\Grpc\Core\Stats::initOnce();
parent::__construct();
}
/**
* Generated from protobuf field <code>string name = 1;</code>
* @return string
*/
public function getName()
{
return $this->name;
}
/**
* Generated from protobuf field <code>string name = 1;</code>
* @param string $var
* @return $this
*/
public function setName($var)
{
GPBUtil::checkString($var, True);
$this->name = $var;
return $this;
}
/**
* Generated from protobuf field <code>uint64 count = 10;</code>
* @return int|string
*/
public function getCount()
{
return $this->readOneof(10);
}
/**
* Generated from protobuf field <code>uint64 count = 10;</code>
* @param int|string $var
* @return $this
*/
public function setCount($var)
{
GPBUtil::checkUint64($var);
$this->writeOneof(10, $var);
return $this;
}
/**
* Generated from protobuf field <code>.grpc.core.Histogram histogram = 11;</code>
* @return \Grpc\Core\Histogram
*/
public function getHistogram()
{
return $this->readOneof(11);
}
/**
* Generated from protobuf field <code>.grpc.core.Histogram histogram = 11;</code>
* @param \Grpc\Core\Histogram $var
* @return $this
*/
public function setHistogram($var)
{
GPBUtil::checkMessage($var, \Grpc\Core\Histogram::class);
$this->writeOneof(11, $var);
return $this;
}
/**
* @return string
*/
public function getValue()
{
return $this->whichOneof("value");
}
}

@ -0,0 +1,49 @@
<?php
# Generated by the protocol buffer compiler. DO NOT EDIT!
# source: src/proto/grpc/core/stats.proto
namespace Grpc\Core;
use Google\Protobuf\Internal\GPBType;
use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* Generated from protobuf message <code>grpc.core.Stats</code>
*/
class Stats extends \Google\Protobuf\Internal\Message
{
/**
* Generated from protobuf field <code>repeated .grpc.core.Metric metrics = 1;</code>
*/
private $metrics;
public function __construct() {
\GPBMetadata\Src\Proto\Grpc\Core\Stats::initOnce();
parent::__construct();
}
/**
* Generated from protobuf field <code>repeated .grpc.core.Metric metrics = 1;</code>
* @return \Google\Protobuf\Internal\RepeatedField
*/
public function getMetrics()
{
return $this->metrics;
}
/**
* Generated from protobuf field <code>repeated .grpc.core.Metric metrics = 1;</code>
* @param \Grpc\Core\Metric[]|\Google\Protobuf\Internal\RepeatedField $var
* @return $this
*/
public function setMetrics($var)
{
$arr = GPBUtil::checkRepeatedField($var, \Google\Protobuf\Internal\GPBType::MESSAGE, \Grpc\Core\Metric::class);
$this->metrics = $arr;
return $this;
}
}

@ -18,17 +18,19 @@
//
// An integration test service that covers all the method signature permutations
// of unary/streaming requests/responses.
namespace Grpc\Testing {
namespace Grpc\Testing;
class BenchmarkServiceClient extends \Grpc\BaseStub {
/**
*/
class BenchmarkServiceClient extends \Grpc\BaseStub {
/**
* @param string $hostname hostname
* @param array $opts channel options
* @param Grpc\Channel $channel (optional) re-use channel object
* @param \Grpc\Channel $channel (optional) re-use channel object
*/
public function __construct($hostname, $opts, $channel = null) {
parent::__construct($hostname, $opts, $channel);
parent::__construct($hostname, $opts, $channel);
}
/**
@ -40,24 +42,62 @@ namespace Grpc\Testing {
*/
public function UnaryCall(\Grpc\Testing\SimpleRequest $argument,
$metadata = [], $options = []) {
return $this->_simpleRequest('/grpc.testing.BenchmarkService/UnaryCall',
$argument,
['\Grpc\Testing\SimpleResponse', 'decode'],
$metadata, $options);
return $this->_simpleRequest('/grpc.testing.BenchmarkService/UnaryCall',
$argument,
['\Grpc\Testing\SimpleResponse', 'decode'],
$metadata, $options);
}
/**
* One request followed by one response.
* The server returns the client payload as-is.
* Repeated sequence of one request followed by one response.
* Should be called streaming ping-pong
* The server returns the client payload as-is on each response
* @param array $metadata metadata
* @param array $options call options
*/
public function StreamingCall($metadata = [], $options = []) {
return $this->_bidiRequest('/grpc.testing.BenchmarkService/StreamingCall',
['\Grpc\Testing\SimpleResponse','decode'],
$metadata, $options);
return $this->_bidiRequest('/grpc.testing.BenchmarkService/StreamingCall',
['\Grpc\Testing\SimpleResponse','decode'],
$metadata, $options);
}
}
/**
* Single-sided unbounded streaming from client to server
* The server returns the client payload as-is once the client does WritesDone
* @param array $metadata metadata
* @param array $options call options
*/
public function StreamingFromClient($metadata = [], $options = []) {
return $this->_clientStreamRequest('/grpc.testing.BenchmarkService/StreamingFromClient',
['\Grpc\Testing\SimpleResponse','decode'],
$metadata, $options);
}
/**
* Single-sided unbounded streaming from server to client
* The server repeatedly returns the client payload as-is
* @param \Grpc\Testing\SimpleRequest $argument input argument
* @param array $metadata metadata
* @param array $options call options
*/
public function StreamingFromServer(\Grpc\Testing\SimpleRequest $argument,
$metadata = [], $options = []) {
return $this->_serverStreamRequest('/grpc.testing.BenchmarkService/StreamingFromServer',
$argument,
['\Grpc\Testing\SimpleResponse', 'decode'],
$metadata, $options);
}
/**
* Two-sided unbounded streaming between server to client
* Both sides send the content of their own choice to the other
* @param array $metadata metadata
* @param array $options call options
*/
public function StreamingBothWays($metadata = [], $options = []) {
return $this->_bidiRequest('/grpc.testing.BenchmarkService/StreamingBothWays',
['\Grpc\Testing\SimpleResponse','decode'],
$metadata, $options);
}
}

@ -9,22 +9,18 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* <pre>
* TODO(dgq): Go back to using well-known types once
* https://github.com/grpc/grpc/issues/6980 has been fixed.
* import "google/protobuf/wrappers.proto";
* </pre>
*
* Protobuf type <code>grpc.testing.BoolValue</code>
* Generated from protobuf message <code>grpc.testing.BoolValue</code>
*/
class BoolValue extends \Google\Protobuf\Internal\Message
{
/**
* <pre>
* The bool value.
* </pre>
*
* <code>bool value = 1;</code>
* Generated from protobuf field <code>bool value = 1;</code>
*/
private $value = false;
@ -34,11 +30,10 @@ class BoolValue extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* The bool value.
* </pre>
*
* <code>bool value = 1;</code>
* Generated from protobuf field <code>bool value = 1;</code>
* @return bool
*/
public function getValue()
{
@ -46,16 +41,18 @@ class BoolValue extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* The bool value.
* </pre>
*
* <code>bool value = 1;</code>
* Generated from protobuf field <code>bool value = 1;</code>
* @param bool $var
* @return $this
*/
public function setValue($var)
{
GPBUtil::checkBool($var);
$this->value = $var;
return $this;
}
}

@ -9,16 +9,16 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* Protobuf type <code>grpc.testing.ByteBufferParams</code>
* Generated from protobuf message <code>grpc.testing.ByteBufferParams</code>
*/
class ByteBufferParams extends \Google\Protobuf\Internal\Message
{
/**
* <code>int32 req_size = 1;</code>
* Generated from protobuf field <code>int32 req_size = 1;</code>
*/
private $req_size = 0;
/**
* <code>int32 resp_size = 2;</code>
* Generated from protobuf field <code>int32 resp_size = 2;</code>
*/
private $resp_size = 0;
@ -28,7 +28,8 @@ class ByteBufferParams extends \Google\Protobuf\Internal\Message
}
/**
* <code>int32 req_size = 1;</code>
* Generated from protobuf field <code>int32 req_size = 1;</code>
* @return int
*/
public function getReqSize()
{
@ -36,16 +37,21 @@ class ByteBufferParams extends \Google\Protobuf\Internal\Message
}
/**
* <code>int32 req_size = 1;</code>
* Generated from protobuf field <code>int32 req_size = 1;</code>
* @param int $var
* @return $this
*/
public function setReqSize($var)
{
GPBUtil::checkInt32($var);
$this->req_size = $var;
return $this;
}
/**
* <code>int32 resp_size = 2;</code>
* Generated from protobuf field <code>int32 resp_size = 2;</code>
* @return int
*/
public function getRespSize()
{
@ -53,12 +59,16 @@ class ByteBufferParams extends \Google\Protobuf\Internal\Message
}
/**
* <code>int32 resp_size = 2;</code>
* Generated from protobuf field <code>int32 resp_size = 2;</code>
* @param int $var
* @return $this
*/
public function setRespSize($var)
{
GPBUtil::checkInt32($var);
$this->resp_size = $var;
return $this;
}
}

@ -9,12 +9,12 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* Protobuf type <code>grpc.testing.ChannelArg</code>
* Generated from protobuf message <code>grpc.testing.ChannelArg</code>
*/
class ChannelArg extends \Google\Protobuf\Internal\Message
{
/**
* <code>string name = 1;</code>
* Generated from protobuf field <code>string name = 1;</code>
*/
private $name = '';
protected $value;
@ -25,7 +25,8 @@ class ChannelArg extends \Google\Protobuf\Internal\Message
}
/**
* <code>string name = 1;</code>
* Generated from protobuf field <code>string name = 1;</code>
* @return string
*/
public function getName()
{
@ -33,16 +34,21 @@ class ChannelArg extends \Google\Protobuf\Internal\Message
}
/**
* <code>string name = 1;</code>
* Generated from protobuf field <code>string name = 1;</code>
* @param string $var
* @return $this
*/
public function setName($var)
{
GPBUtil::checkString($var, True);
$this->name = $var;
return $this;
}
/**
* <code>string str_value = 2;</code>
* Generated from protobuf field <code>string str_value = 2;</code>
* @return string
*/
public function getStrValue()
{
@ -50,16 +56,21 @@ class ChannelArg extends \Google\Protobuf\Internal\Message
}
/**
* <code>string str_value = 2;</code>
* Generated from protobuf field <code>string str_value = 2;</code>
* @param string $var
* @return $this
*/
public function setStrValue($var)
{
GPBUtil::checkString($var, True);
$this->writeOneof(2, $var);
return $this;
}
/**
* <code>int32 int_value = 3;</code>
* Generated from protobuf field <code>int32 int_value = 3;</code>
* @return int
*/
public function getIntValue()
{
@ -67,14 +78,21 @@ class ChannelArg extends \Google\Protobuf\Internal\Message
}
/**
* <code>int32 int_value = 3;</code>
* Generated from protobuf field <code>int32 int_value = 3;</code>
* @param int $var
* @return $this
*/
public function setIntValue($var)
{
GPBUtil::checkInt32($var);
$this->writeOneof(3, $var);
return $this;
}
/**
* @return string
*/
public function getValue()
{
return $this->whichOneof("value");

@ -9,7 +9,7 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* Protobuf type <code>grpc.testing.ClientArgs</code>
* Generated from protobuf message <code>grpc.testing.ClientArgs</code>
*/
class ClientArgs extends \Google\Protobuf\Internal\Message
{
@ -21,7 +21,8 @@ class ClientArgs extends \Google\Protobuf\Internal\Message
}
/**
* <code>.grpc.testing.ClientConfig setup = 1;</code>
* Generated from protobuf field <code>.grpc.testing.ClientConfig setup = 1;</code>
* @return \Grpc\Testing\ClientConfig
*/
public function getSetup()
{
@ -29,16 +30,21 @@ class ClientArgs extends \Google\Protobuf\Internal\Message
}
/**
* <code>.grpc.testing.ClientConfig setup = 1;</code>
* Generated from protobuf field <code>.grpc.testing.ClientConfig setup = 1;</code>
* @param \Grpc\Testing\ClientConfig $var
* @return $this
*/
public function setSetup(&$var)
public function setSetup($var)
{
GPBUtil::checkMessage($var, \Grpc\Testing\ClientConfig::class);
$this->writeOneof(1, $var);
return $this;
}
/**
* <code>.grpc.testing.Mark mark = 2;</code>
* Generated from protobuf field <code>.grpc.testing.Mark mark = 2;</code>
* @return \Grpc\Testing\Mark
*/
public function getMark()
{
@ -46,14 +52,21 @@ class ClientArgs extends \Google\Protobuf\Internal\Message
}
/**
* <code>.grpc.testing.Mark mark = 2;</code>
* Generated from protobuf field <code>.grpc.testing.Mark mark = 2;</code>
* @param \Grpc\Testing\Mark $var
* @return $this
*/
public function setMark(&$var)
public function setMark($var)
{
GPBUtil::checkMessage($var, \Grpc\Testing\Mark::class);
$this->writeOneof(2, $var);
return $this;
}
/**
* @return string
*/
public function getArgtype()
{
return $this->whichOneof("argtype");

@ -9,96 +9,94 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* Protobuf type <code>grpc.testing.ClientConfig</code>
* Generated from protobuf message <code>grpc.testing.ClientConfig</code>
*/
class ClientConfig extends \Google\Protobuf\Internal\Message
{
/**
* <pre>
* List of targets to connect to. At least one target needs to be specified.
* </pre>
*
* <code>repeated string server_targets = 1;</code>
* Generated from protobuf field <code>repeated string server_targets = 1;</code>
*/
private $server_targets;
/**
* <code>.grpc.testing.ClientType client_type = 2;</code>
* Generated from protobuf field <code>.grpc.testing.ClientType client_type = 2;</code>
*/
private $client_type = 0;
/**
* <code>.grpc.testing.SecurityParams security_params = 3;</code>
* Generated from protobuf field <code>.grpc.testing.SecurityParams security_params = 3;</code>
*/
private $security_params = null;
/**
* <pre>
* How many concurrent RPCs to start for each channel.
* For synchronous client, use a separate thread for each outstanding RPC.
* </pre>
*
* <code>int32 outstanding_rpcs_per_channel = 4;</code>
* Generated from protobuf field <code>int32 outstanding_rpcs_per_channel = 4;</code>
*/
private $outstanding_rpcs_per_channel = 0;
/**
* <pre>
* Number of independent client channels to create.
* i-th channel will connect to server_target[i % server_targets.size()]
* </pre>
*
* <code>int32 client_channels = 5;</code>
* Generated from protobuf field <code>int32 client_channels = 5;</code>
*/
private $client_channels = 0;
/**
* <pre>
* Only for async client. Number of threads to use to start/manage RPCs.
* </pre>
*
* <code>int32 async_client_threads = 7;</code>
* Generated from protobuf field <code>int32 async_client_threads = 7;</code>
*/
private $async_client_threads = 0;
/**
* <code>.grpc.testing.RpcType rpc_type = 8;</code>
* Generated from protobuf field <code>.grpc.testing.RpcType rpc_type = 8;</code>
*/
private $rpc_type = 0;
/**
* <pre>
* The requested load for the entire client (aggregated over all the threads).
* </pre>
*
* <code>.grpc.testing.LoadParams load_params = 10;</code>
* Generated from protobuf field <code>.grpc.testing.LoadParams load_params = 10;</code>
*/
private $load_params = null;
/**
* <code>.grpc.testing.PayloadConfig payload_config = 11;</code>
* Generated from protobuf field <code>.grpc.testing.PayloadConfig payload_config = 11;</code>
*/
private $payload_config = null;
/**
* <code>.grpc.testing.HistogramParams histogram_params = 12;</code>
* Generated from protobuf field <code>.grpc.testing.HistogramParams histogram_params = 12;</code>
*/
private $histogram_params = null;
/**
* <pre>
* Specify the cores we should run the client on, if desired
* </pre>
*
* <code>repeated int32 core_list = 13;</code>
* Generated from protobuf field <code>repeated int32 core_list = 13;</code>
*/
private $core_list;
/**
* <code>int32 core_limit = 14;</code>
* Generated from protobuf field <code>int32 core_limit = 14;</code>
*/
private $core_limit = 0;
/**
* <pre>
* If we use an OTHER_CLIENT client_type, this string gives more detail
* </pre>
*
* <code>string other_client_api = 15;</code>
* Generated from protobuf field <code>string other_client_api = 15;</code>
*/
private $other_client_api = '';
/**
* <code>repeated .grpc.testing.ChannelArg channel_args = 16;</code>
* Generated from protobuf field <code>repeated .grpc.testing.ChannelArg channel_args = 16;</code>
*/
private $channel_args;
/**
* Number of threads that share each completion queue
*
* Generated from protobuf field <code>int32 threads_per_cq = 17;</code>
*/
private $threads_per_cq = 0;
/**
* Number of messages on a stream before it gets finished/restarted
*
* Generated from protobuf field <code>int32 messages_per_stream = 18;</code>
*/
private $messages_per_stream = 0;
public function __construct() {
\GPBMetadata\Src\Proto\Grpc\Testing\Control::initOnce();
@ -106,11 +104,10 @@ class ClientConfig extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* List of targets to connect to. At least one target needs to be specified.
* </pre>
*
* <code>repeated string server_targets = 1;</code>
* Generated from protobuf field <code>repeated string server_targets = 1;</code>
* @return \Google\Protobuf\Internal\RepeatedField
*/
public function getServerTargets()
{
@ -118,20 +115,23 @@ class ClientConfig extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* List of targets to connect to. At least one target needs to be specified.
* </pre>
*
* <code>repeated string server_targets = 1;</code>
* Generated from protobuf field <code>repeated string server_targets = 1;</code>
* @param string[]|\Google\Protobuf\Internal\RepeatedField $var
* @return $this
*/
public function setServerTargets(&$var)
public function setServerTargets($var)
{
GPBUtil::checkRepeatedField($var, \Google\Protobuf\Internal\GPBType::STRING);
$this->server_targets = $var;
$arr = GPBUtil::checkRepeatedField($var, \Google\Protobuf\Internal\GPBType::STRING);
$this->server_targets = $arr;
return $this;
}
/**
* <code>.grpc.testing.ClientType client_type = 2;</code>
* Generated from protobuf field <code>.grpc.testing.ClientType client_type = 2;</code>
* @return int
*/
public function getClientType()
{
@ -139,16 +139,21 @@ class ClientConfig extends \Google\Protobuf\Internal\Message
}
/**
* <code>.grpc.testing.ClientType client_type = 2;</code>
* Generated from protobuf field <code>.grpc.testing.ClientType client_type = 2;</code>
* @param int $var
* @return $this
*/
public function setClientType($var)
{
GPBUtil::checkEnum($var, \Grpc\Testing\ClientType::class);
$this->client_type = $var;
return $this;
}
/**
* <code>.grpc.testing.SecurityParams security_params = 3;</code>
* Generated from protobuf field <code>.grpc.testing.SecurityParams security_params = 3;</code>
* @return \Grpc\Testing\SecurityParams
*/
public function getSecurityParams()
{
@ -156,21 +161,24 @@ class ClientConfig extends \Google\Protobuf\Internal\Message
}
/**
* <code>.grpc.testing.SecurityParams security_params = 3;</code>
* Generated from protobuf field <code>.grpc.testing.SecurityParams security_params = 3;</code>
* @param \Grpc\Testing\SecurityParams $var
* @return $this
*/
public function setSecurityParams(&$var)
public function setSecurityParams($var)
{
GPBUtil::checkMessage($var, \Grpc\Testing\SecurityParams::class);
$this->security_params = $var;
return $this;
}
/**
* <pre>
* How many concurrent RPCs to start for each channel.
* For synchronous client, use a separate thread for each outstanding RPC.
* </pre>
*
* <code>int32 outstanding_rpcs_per_channel = 4;</code>
* Generated from protobuf field <code>int32 outstanding_rpcs_per_channel = 4;</code>
* @return int
*/
public function getOutstandingRpcsPerChannel()
{
@ -178,26 +186,27 @@ class ClientConfig extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* How many concurrent RPCs to start for each channel.
* For synchronous client, use a separate thread for each outstanding RPC.
* </pre>
*
* <code>int32 outstanding_rpcs_per_channel = 4;</code>
* Generated from protobuf field <code>int32 outstanding_rpcs_per_channel = 4;</code>
* @param int $var
* @return $this
*/
public function setOutstandingRpcsPerChannel($var)
{
GPBUtil::checkInt32($var);
$this->outstanding_rpcs_per_channel = $var;
return $this;
}
/**
* <pre>
* Number of independent client channels to create.
* i-th channel will connect to server_target[i % server_targets.size()]
* </pre>
*
* <code>int32 client_channels = 5;</code>
* Generated from protobuf field <code>int32 client_channels = 5;</code>
* @return int
*/
public function getClientChannels()
{
@ -205,25 +214,26 @@ class ClientConfig extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Number of independent client channels to create.
* i-th channel will connect to server_target[i % server_targets.size()]
* </pre>
*
* <code>int32 client_channels = 5;</code>
* Generated from protobuf field <code>int32 client_channels = 5;</code>
* @param int $var
* @return $this
*/
public function setClientChannels($var)
{
GPBUtil::checkInt32($var);
$this->client_channels = $var;
return $this;
}
/**
* <pre>
* Only for async client. Number of threads to use to start/manage RPCs.
* </pre>
*
* <code>int32 async_client_threads = 7;</code>
* Generated from protobuf field <code>int32 async_client_threads = 7;</code>
* @return int
*/
public function getAsyncClientThreads()
{
@ -231,20 +241,23 @@ class ClientConfig extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Only for async client. Number of threads to use to start/manage RPCs.
* </pre>
*
* <code>int32 async_client_threads = 7;</code>
* Generated from protobuf field <code>int32 async_client_threads = 7;</code>
* @param int $var
* @return $this
*/
public function setAsyncClientThreads($var)
{
GPBUtil::checkInt32($var);
$this->async_client_threads = $var;
return $this;
}
/**
* <code>.grpc.testing.RpcType rpc_type = 8;</code>
* Generated from protobuf field <code>.grpc.testing.RpcType rpc_type = 8;</code>
* @return int
*/
public function getRpcType()
{
@ -252,20 +265,23 @@ class ClientConfig extends \Google\Protobuf\Internal\Message
}
/**
* <code>.grpc.testing.RpcType rpc_type = 8;</code>
* Generated from protobuf field <code>.grpc.testing.RpcType rpc_type = 8;</code>
* @param int $var
* @return $this
*/
public function setRpcType($var)
{
GPBUtil::checkEnum($var, \Grpc\Testing\RpcType::class);
$this->rpc_type = $var;
return $this;
}
/**
* <pre>
* The requested load for the entire client (aggregated over all the threads).
* </pre>
*
* <code>.grpc.testing.LoadParams load_params = 10;</code>
* Generated from protobuf field <code>.grpc.testing.LoadParams load_params = 10;</code>
* @return \Grpc\Testing\LoadParams
*/
public function getLoadParams()
{
@ -273,20 +289,23 @@ class ClientConfig extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* The requested load for the entire client (aggregated over all the threads).
* </pre>
*
* <code>.grpc.testing.LoadParams load_params = 10;</code>
* Generated from protobuf field <code>.grpc.testing.LoadParams load_params = 10;</code>
* @param \Grpc\Testing\LoadParams $var
* @return $this
*/
public function setLoadParams(&$var)
public function setLoadParams($var)
{
GPBUtil::checkMessage($var, \Grpc\Testing\LoadParams::class);
$this->load_params = $var;
return $this;
}
/**
* <code>.grpc.testing.PayloadConfig payload_config = 11;</code>
* Generated from protobuf field <code>.grpc.testing.PayloadConfig payload_config = 11;</code>
* @return \Grpc\Testing\PayloadConfig
*/
public function getPayloadConfig()
{
@ -294,16 +313,21 @@ class ClientConfig extends \Google\Protobuf\Internal\Message
}
/**
* <code>.grpc.testing.PayloadConfig payload_config = 11;</code>
* Generated from protobuf field <code>.grpc.testing.PayloadConfig payload_config = 11;</code>
* @param \Grpc\Testing\PayloadConfig $var
* @return $this
*/
public function setPayloadConfig(&$var)
public function setPayloadConfig($var)
{
GPBUtil::checkMessage($var, \Grpc\Testing\PayloadConfig::class);
$this->payload_config = $var;
return $this;
}
/**
* <code>.grpc.testing.HistogramParams histogram_params = 12;</code>
* Generated from protobuf field <code>.grpc.testing.HistogramParams histogram_params = 12;</code>
* @return \Grpc\Testing\HistogramParams
*/
public function getHistogramParams()
{
@ -311,20 +335,23 @@ class ClientConfig extends \Google\Protobuf\Internal\Message
}
/**
* <code>.grpc.testing.HistogramParams histogram_params = 12;</code>
* Generated from protobuf field <code>.grpc.testing.HistogramParams histogram_params = 12;</code>
* @param \Grpc\Testing\HistogramParams $var
* @return $this
*/
public function setHistogramParams(&$var)
public function setHistogramParams($var)
{
GPBUtil::checkMessage($var, \Grpc\Testing\HistogramParams::class);
$this->histogram_params = $var;
return $this;
}
/**
* <pre>
* Specify the cores we should run the client on, if desired
* </pre>
*
* <code>repeated int32 core_list = 13;</code>
* Generated from protobuf field <code>repeated int32 core_list = 13;</code>
* @return \Google\Protobuf\Internal\RepeatedField
*/
public function getCoreList()
{
@ -332,20 +359,23 @@ class ClientConfig extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Specify the cores we should run the client on, if desired
* </pre>
*
* <code>repeated int32 core_list = 13;</code>
* Generated from protobuf field <code>repeated int32 core_list = 13;</code>
* @param int[]|\Google\Protobuf\Internal\RepeatedField $var
* @return $this
*/
public function setCoreList(&$var)
public function setCoreList($var)
{
GPBUtil::checkRepeatedField($var, \Google\Protobuf\Internal\GPBType::INT32);
$this->core_list = $var;
$arr = GPBUtil::checkRepeatedField($var, \Google\Protobuf\Internal\GPBType::INT32);
$this->core_list = $arr;
return $this;
}
/**
* <code>int32 core_limit = 14;</code>
* Generated from protobuf field <code>int32 core_limit = 14;</code>
* @return int
*/
public function getCoreLimit()
{
@ -353,20 +383,23 @@ class ClientConfig extends \Google\Protobuf\Internal\Message
}
/**
* <code>int32 core_limit = 14;</code>
* Generated from protobuf field <code>int32 core_limit = 14;</code>
* @param int $var
* @return $this
*/
public function setCoreLimit($var)
{
GPBUtil::checkInt32($var);
$this->core_limit = $var;
return $this;
}
/**
* <pre>
* If we use an OTHER_CLIENT client_type, this string gives more detail
* </pre>
*
* <code>string other_client_api = 15;</code>
* Generated from protobuf field <code>string other_client_api = 15;</code>
* @return string
*/
public function getOtherClientApi()
{
@ -374,20 +407,23 @@ class ClientConfig extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* If we use an OTHER_CLIENT client_type, this string gives more detail
* </pre>
*
* <code>string other_client_api = 15;</code>
* Generated from protobuf field <code>string other_client_api = 15;</code>
* @param string $var
* @return $this
*/
public function setOtherClientApi($var)
{
GPBUtil::checkString($var, True);
$this->other_client_api = $var;
return $this;
}
/**
* <code>repeated .grpc.testing.ChannelArg channel_args = 16;</code>
* Generated from protobuf field <code>repeated .grpc.testing.ChannelArg channel_args = 16;</code>
* @return \Google\Protobuf\Internal\RepeatedField
*/
public function getChannelArgs()
{
@ -395,12 +431,68 @@ class ClientConfig extends \Google\Protobuf\Internal\Message
}
/**
* <code>repeated .grpc.testing.ChannelArg channel_args = 16;</code>
* Generated from protobuf field <code>repeated .grpc.testing.ChannelArg channel_args = 16;</code>
* @param \Grpc\Testing\ChannelArg[]|\Google\Protobuf\Internal\RepeatedField $var
* @return $this
*/
public function setChannelArgs($var)
{
$arr = GPBUtil::checkRepeatedField($var, \Google\Protobuf\Internal\GPBType::MESSAGE, \Grpc\Testing\ChannelArg::class);
$this->channel_args = $arr;
return $this;
}
/**
* Number of threads that share each completion queue
*
* Generated from protobuf field <code>int32 threads_per_cq = 17;</code>
* @return int
*/
public function getThreadsPerCq()
{
return $this->threads_per_cq;
}
/**
* Number of threads that share each completion queue
*
* Generated from protobuf field <code>int32 threads_per_cq = 17;</code>
* @param int $var
* @return $this
*/
public function setThreadsPerCq($var)
{
GPBUtil::checkInt32($var);
$this->threads_per_cq = $var;
return $this;
}
/**
* Number of messages on a stream before it gets finished/restarted
*
* Generated from protobuf field <code>int32 messages_per_stream = 18;</code>
* @return int
*/
public function getMessagesPerStream()
{
return $this->messages_per_stream;
}
/**
* Number of messages on a stream before it gets finished/restarted
*
* Generated from protobuf field <code>int32 messages_per_stream = 18;</code>
* @param int $var
* @return $this
*/
public function setChannelArgs(&$var)
public function setMessagesPerStream($var)
{
GPBUtil::checkRepeatedField($var, \Google\Protobuf\Internal\GPBType::MESSAGE, \Grpc\Testing\ChannelArg::class);
$this->channel_args = $var;
GPBUtil::checkInt32($var);
$this->messages_per_stream = $var;
return $this;
}
}

@ -9,42 +9,48 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* Protobuf type <code>grpc.testing.ClientStats</code>
* Generated from protobuf message <code>grpc.testing.ClientStats</code>
*/
class ClientStats extends \Google\Protobuf\Internal\Message
{
/**
* <pre>
* Latency histogram. Data points are in nanoseconds.
* </pre>
*
* <code>.grpc.testing.HistogramData latencies = 1;</code>
* Generated from protobuf field <code>.grpc.testing.HistogramData latencies = 1;</code>
*/
private $latencies = null;
/**
* <pre>
* See ServerStats for details.
* </pre>
*
* <code>double time_elapsed = 2;</code>
* Generated from protobuf field <code>double time_elapsed = 2;</code>
*/
private $time_elapsed = 0.0;
/**
* <code>double time_user = 3;</code>
* Generated from protobuf field <code>double time_user = 3;</code>
*/
private $time_user = 0.0;
/**
* <code>double time_system = 4;</code>
* Generated from protobuf field <code>double time_system = 4;</code>
*/
private $time_system = 0.0;
/**
* <pre>
* Number of failed requests (one row per status code seen)
* </pre>
*
* <code>repeated .grpc.testing.RequestResultCount request_results = 5;</code>
* Generated from protobuf field <code>repeated .grpc.testing.RequestResultCount request_results = 5;</code>
*/
private $request_results;
/**
* Number of polls called inside completion queue
*
* Generated from protobuf field <code>uint64 cq_poll_count = 6;</code>
*/
private $cq_poll_count = 0;
/**
* Core library stats
*
* Generated from protobuf field <code>.grpc.core.Stats core_stats = 7;</code>
*/
private $core_stats = null;
public function __construct() {
\GPBMetadata\Src\Proto\Grpc\Testing\Stats::initOnce();
@ -52,11 +58,10 @@ class ClientStats extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Latency histogram. Data points are in nanoseconds.
* </pre>
*
* <code>.grpc.testing.HistogramData latencies = 1;</code>
* Generated from protobuf field <code>.grpc.testing.HistogramData latencies = 1;</code>
* @return \Grpc\Testing\HistogramData
*/
public function getLatencies()
{
@ -64,24 +69,25 @@ class ClientStats extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Latency histogram. Data points are in nanoseconds.
* </pre>
*
* <code>.grpc.testing.HistogramData latencies = 1;</code>
* Generated from protobuf field <code>.grpc.testing.HistogramData latencies = 1;</code>
* @param \Grpc\Testing\HistogramData $var
* @return $this
*/
public function setLatencies(&$var)
public function setLatencies($var)
{
GPBUtil::checkMessage($var, \Grpc\Testing\HistogramData::class);
$this->latencies = $var;
return $this;
}
/**
* <pre>
* See ServerStats for details.
* </pre>
*
* <code>double time_elapsed = 2;</code>
* Generated from protobuf field <code>double time_elapsed = 2;</code>
* @return float
*/
public function getTimeElapsed()
{
@ -89,20 +95,23 @@ class ClientStats extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* See ServerStats for details.
* </pre>
*
* <code>double time_elapsed = 2;</code>
* Generated from protobuf field <code>double time_elapsed = 2;</code>
* @param float $var
* @return $this
*/
public function setTimeElapsed($var)
{
GPBUtil::checkDouble($var);
$this->time_elapsed = $var;
return $this;
}
/**
* <code>double time_user = 3;</code>
* Generated from protobuf field <code>double time_user = 3;</code>
* @return float
*/
public function getTimeUser()
{
@ -110,16 +119,21 @@ class ClientStats extends \Google\Protobuf\Internal\Message
}
/**
* <code>double time_user = 3;</code>
* Generated from protobuf field <code>double time_user = 3;</code>
* @param float $var
* @return $this
*/
public function setTimeUser($var)
{
GPBUtil::checkDouble($var);
$this->time_user = $var;
return $this;
}
/**
* <code>double time_system = 4;</code>
* Generated from protobuf field <code>double time_system = 4;</code>
* @return float
*/
public function getTimeSystem()
{
@ -127,20 +141,23 @@ class ClientStats extends \Google\Protobuf\Internal\Message
}
/**
* <code>double time_system = 4;</code>
* Generated from protobuf field <code>double time_system = 4;</code>
* @param float $var
* @return $this
*/
public function setTimeSystem($var)
{
GPBUtil::checkDouble($var);
$this->time_system = $var;
return $this;
}
/**
* <pre>
* Number of failed requests (one row per status code seen)
* </pre>
*
* <code>repeated .grpc.testing.RequestResultCount request_results = 5;</code>
* Generated from protobuf field <code>repeated .grpc.testing.RequestResultCount request_results = 5;</code>
* @return \Google\Protobuf\Internal\RepeatedField
*/
public function getRequestResults()
{
@ -148,16 +165,70 @@ class ClientStats extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Number of failed requests (one row per status code seen)
* </pre>
*
* <code>repeated .grpc.testing.RequestResultCount request_results = 5;</code>
* Generated from protobuf field <code>repeated .grpc.testing.RequestResultCount request_results = 5;</code>
* @param \Grpc\Testing\RequestResultCount[]|\Google\Protobuf\Internal\RepeatedField $var
* @return $this
*/
public function setRequestResults(&$var)
public function setRequestResults($var)
{
GPBUtil::checkRepeatedField($var, \Google\Protobuf\Internal\GPBType::MESSAGE, \Grpc\Testing\RequestResultCount::class);
$this->request_results = $var;
$arr = GPBUtil::checkRepeatedField($var, \Google\Protobuf\Internal\GPBType::MESSAGE, \Grpc\Testing\RequestResultCount::class);
$this->request_results = $arr;
return $this;
}
/**
* Number of polls called inside completion queue
*
* Generated from protobuf field <code>uint64 cq_poll_count = 6;</code>
* @return int|string
*/
public function getCqPollCount()
{
return $this->cq_poll_count;
}
/**
* Number of polls called inside completion queue
*
* Generated from protobuf field <code>uint64 cq_poll_count = 6;</code>
* @param int|string $var
* @return $this
*/
public function setCqPollCount($var)
{
GPBUtil::checkUint64($var);
$this->cq_poll_count = $var;
return $this;
}
/**
* Core library stats
*
* Generated from protobuf field <code>.grpc.core.Stats core_stats = 7;</code>
* @return \Grpc\Core\Stats
*/
public function getCoreStats()
{
return $this->core_stats;
}
/**
* Core library stats
*
* Generated from protobuf field <code>.grpc.core.Stats core_stats = 7;</code>
* @param \Grpc\Core\Stats $var
* @return $this
*/
public function setCoreStats($var)
{
GPBUtil::checkMessage($var, \Grpc\Core\Stats::class);
$this->core_stats = $var;
return $this;
}
}

@ -9,12 +9,12 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* Protobuf type <code>grpc.testing.ClientStatus</code>
* Generated from protobuf message <code>grpc.testing.ClientStatus</code>
*/
class ClientStatus extends \Google\Protobuf\Internal\Message
{
/**
* <code>.grpc.testing.ClientStats stats = 1;</code>
* Generated from protobuf field <code>.grpc.testing.ClientStats stats = 1;</code>
*/
private $stats = null;
@ -24,7 +24,8 @@ class ClientStatus extends \Google\Protobuf\Internal\Message
}
/**
* <code>.grpc.testing.ClientStats stats = 1;</code>
* Generated from protobuf field <code>.grpc.testing.ClientStats stats = 1;</code>
* @return \Grpc\Testing\ClientStats
*/
public function getStats()
{
@ -32,12 +33,16 @@ class ClientStatus extends \Google\Protobuf\Internal\Message
}
/**
* <code>.grpc.testing.ClientStats stats = 1;</code>
* Generated from protobuf field <code>.grpc.testing.ClientStats stats = 1;</code>
* @param \Grpc\Testing\ClientStats $var
* @return $this
*/
public function setStats(&$var)
public function setStats($var)
{
GPBUtil::checkMessage($var, \Grpc\Testing\ClientStats::class);
$this->stats = $var;
return $this;
}
}

@ -5,29 +5,25 @@
namespace Grpc\Testing;
/**
* Protobuf enum <code>grpc.testing.ClientType</code>
* Protobuf enum <code>Grpc\Testing\ClientType</code>
*/
class ClientType
{
/**
* <pre>
* Many languages support a basic distinction between using
* sync or async client, and this allows the specification
* </pre>
*
* <code>SYNC_CLIENT = 0;</code>
* Generated from protobuf enum <code>SYNC_CLIENT = 0;</code>
*/
const SYNC_CLIENT = 0;
/**
* <code>ASYNC_CLIENT = 1;</code>
* Generated from protobuf enum <code>ASYNC_CLIENT = 1;</code>
*/
const ASYNC_CLIENT = 1;
/**
* <pre>
* used for some language-specific variants
* </pre>
*
* <code>OTHER_CLIENT = 2;</code>
* Generated from protobuf enum <code>OTHER_CLIENT = 2;</code>
*/
const OTHER_CLIENT = 2;
}

@ -9,12 +9,10 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* <pre>
* Once an RPC finishes, immediately start a new one.
* No configuration parameters needed.
* </pre>
*
* Protobuf type <code>grpc.testing.ClosedLoopParams</code>
* Generated from protobuf message <code>grpc.testing.ClosedLoopParams</code>
*/
class ClosedLoopParams extends \Google\Protobuf\Internal\Message
{

@ -9,12 +9,10 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* <pre>
* TODO (vpai): Fill this in once the details of complex, representative
* protos are decided
* </pre>
*
* Protobuf type <code>grpc.testing.ComplexProtoParams</code>
* Generated from protobuf message <code>grpc.testing.ComplexProtoParams</code>
*/
class ComplexProtoParams extends \Google\Protobuf\Internal\Message
{

@ -9,7 +9,7 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* Protobuf type <code>grpc.testing.CoreRequest</code>
* Generated from protobuf message <code>grpc.testing.CoreRequest</code>
*/
class CoreRequest extends \Google\Protobuf\Internal\Message
{

@ -9,16 +9,14 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* Protobuf type <code>grpc.testing.CoreResponse</code>
* Generated from protobuf message <code>grpc.testing.CoreResponse</code>
*/
class CoreResponse extends \Google\Protobuf\Internal\Message
{
/**
* <pre>
* Number of cores available on the server
* </pre>
*
* <code>int32 cores = 1;</code>
* Generated from protobuf field <code>int32 cores = 1;</code>
*/
private $cores = 0;
@ -28,11 +26,10 @@ class CoreResponse extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Number of cores available on the server
* </pre>
*
* <code>int32 cores = 1;</code>
* Generated from protobuf field <code>int32 cores = 1;</code>
* @return int
*/
public function getCores()
{
@ -40,16 +37,18 @@ class CoreResponse extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Number of cores available on the server
* </pre>
*
* <code>int32 cores = 1;</code>
* Generated from protobuf field <code>int32 cores = 1;</code>
* @param int $var
* @return $this
*/
public function setCores($var)
{
GPBUtil::checkInt32($var);
$this->cores = $var;
return $this;
}
}

@ -9,21 +9,19 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* <pre>
* A protobuf representation for grpc status. This is used by test
* clients to specify a status that the server should attempt to return.
* </pre>
*
* Protobuf type <code>grpc.testing.EchoStatus</code>
* Generated from protobuf message <code>grpc.testing.EchoStatus</code>
*/
class EchoStatus extends \Google\Protobuf\Internal\Message
{
/**
* <code>int32 code = 1;</code>
* Generated from protobuf field <code>int32 code = 1;</code>
*/
private $code = 0;
/**
* <code>string message = 2;</code>
* Generated from protobuf field <code>string message = 2;</code>
*/
private $message = '';
@ -33,7 +31,8 @@ class EchoStatus extends \Google\Protobuf\Internal\Message
}
/**
* <code>int32 code = 1;</code>
* Generated from protobuf field <code>int32 code = 1;</code>
* @return int
*/
public function getCode()
{
@ -41,16 +40,21 @@ class EchoStatus extends \Google\Protobuf\Internal\Message
}
/**
* <code>int32 code = 1;</code>
* Generated from protobuf field <code>int32 code = 1;</code>
* @param int $var
* @return $this
*/
public function setCode($var)
{
GPBUtil::checkInt32($var);
$this->code = $var;
return $this;
}
/**
* <code>string message = 2;</code>
* Generated from protobuf field <code>string message = 2;</code>
* @return string
*/
public function getMessage()
{
@ -58,12 +62,16 @@ class EchoStatus extends \Google\Protobuf\Internal\Message
}
/**
* <code>string message = 2;</code>
* Generated from protobuf field <code>string message = 2;</code>
* @param string $var
* @return $this
*/
public function setMessage($var)
{
GPBUtil::checkString($var, True);
$this->message = $var;
return $this;
}
}

@ -9,36 +9,34 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* <pre>
* Histogram data based on grpc/support/histogram.c
* </pre>
*
* Protobuf type <code>grpc.testing.HistogramData</code>
* Generated from protobuf message <code>grpc.testing.HistogramData</code>
*/
class HistogramData extends \Google\Protobuf\Internal\Message
{
/**
* <code>repeated uint32 bucket = 1;</code>
* Generated from protobuf field <code>repeated uint32 bucket = 1;</code>
*/
private $bucket;
/**
* <code>double min_seen = 2;</code>
* Generated from protobuf field <code>double min_seen = 2;</code>
*/
private $min_seen = 0.0;
/**
* <code>double max_seen = 3;</code>
* Generated from protobuf field <code>double max_seen = 3;</code>
*/
private $max_seen = 0.0;
/**
* <code>double sum = 4;</code>
* Generated from protobuf field <code>double sum = 4;</code>
*/
private $sum = 0.0;
/**
* <code>double sum_of_squares = 5;</code>
* Generated from protobuf field <code>double sum_of_squares = 5;</code>
*/
private $sum_of_squares = 0.0;
/**
* <code>double count = 6;</code>
* Generated from protobuf field <code>double count = 6;</code>
*/
private $count = 0.0;
@ -48,7 +46,8 @@ class HistogramData extends \Google\Protobuf\Internal\Message
}
/**
* <code>repeated uint32 bucket = 1;</code>
* Generated from protobuf field <code>repeated uint32 bucket = 1;</code>
* @return \Google\Protobuf\Internal\RepeatedField
*/
public function getBucket()
{
@ -56,16 +55,21 @@ class HistogramData extends \Google\Protobuf\Internal\Message
}
/**
* <code>repeated uint32 bucket = 1;</code>
* Generated from protobuf field <code>repeated uint32 bucket = 1;</code>
* @param int[]|\Google\Protobuf\Internal\RepeatedField $var
* @return $this
*/
public function setBucket(&$var)
public function setBucket($var)
{
GPBUtil::checkRepeatedField($var, \Google\Protobuf\Internal\GPBType::UINT32);
$this->bucket = $var;
$arr = GPBUtil::checkRepeatedField($var, \Google\Protobuf\Internal\GPBType::UINT32);
$this->bucket = $arr;
return $this;
}
/**
* <code>double min_seen = 2;</code>
* Generated from protobuf field <code>double min_seen = 2;</code>
* @return float
*/
public function getMinSeen()
{
@ -73,16 +77,21 @@ class HistogramData extends \Google\Protobuf\Internal\Message
}
/**
* <code>double min_seen = 2;</code>
* Generated from protobuf field <code>double min_seen = 2;</code>
* @param float $var
* @return $this
*/
public function setMinSeen($var)
{
GPBUtil::checkDouble($var);
$this->min_seen = $var;
return $this;
}
/**
* <code>double max_seen = 3;</code>
* Generated from protobuf field <code>double max_seen = 3;</code>
* @return float
*/
public function getMaxSeen()
{
@ -90,16 +99,21 @@ class HistogramData extends \Google\Protobuf\Internal\Message
}
/**
* <code>double max_seen = 3;</code>
* Generated from protobuf field <code>double max_seen = 3;</code>
* @param float $var
* @return $this
*/
public function setMaxSeen($var)
{
GPBUtil::checkDouble($var);
$this->max_seen = $var;
return $this;
}
/**
* <code>double sum = 4;</code>
* Generated from protobuf field <code>double sum = 4;</code>
* @return float
*/
public function getSum()
{
@ -107,16 +121,21 @@ class HistogramData extends \Google\Protobuf\Internal\Message
}
/**
* <code>double sum = 4;</code>
* Generated from protobuf field <code>double sum = 4;</code>
* @param float $var
* @return $this
*/
public function setSum($var)
{
GPBUtil::checkDouble($var);
$this->sum = $var;
return $this;
}
/**
* <code>double sum_of_squares = 5;</code>
* Generated from protobuf field <code>double sum_of_squares = 5;</code>
* @return float
*/
public function getSumOfSquares()
{
@ -124,16 +143,21 @@ class HistogramData extends \Google\Protobuf\Internal\Message
}
/**
* <code>double sum_of_squares = 5;</code>
* Generated from protobuf field <code>double sum_of_squares = 5;</code>
* @param float $var
* @return $this
*/
public function setSumOfSquares($var)
{
GPBUtil::checkDouble($var);
$this->sum_of_squares = $var;
return $this;
}
/**
* <code>double count = 6;</code>
* Generated from protobuf field <code>double count = 6;</code>
* @return float
*/
public function getCount()
{
@ -141,12 +165,16 @@ class HistogramData extends \Google\Protobuf\Internal\Message
}
/**
* <code>double count = 6;</code>
* Generated from protobuf field <code>double count = 6;</code>
* @param float $var
* @return $this
*/
public function setCount($var)
{
GPBUtil::checkDouble($var);
$this->count = $var;
return $this;
}
}

@ -9,28 +9,22 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* <pre>
* Histogram params based on grpc/support/histogram.c
* </pre>
*
* Protobuf type <code>grpc.testing.HistogramParams</code>
* Generated from protobuf message <code>grpc.testing.HistogramParams</code>
*/
class HistogramParams extends \Google\Protobuf\Internal\Message
{
/**
* <pre>
* first bucket is [0, 1 + resolution)
* </pre>
*
* <code>double resolution = 1;</code>
* Generated from protobuf field <code>double resolution = 1;</code>
*/
private $resolution = 0.0;
/**
* <pre>
* use enough buckets to allow this value
* </pre>
*
* <code>double max_possible = 2;</code>
* Generated from protobuf field <code>double max_possible = 2;</code>
*/
private $max_possible = 0.0;
@ -40,11 +34,10 @@ class HistogramParams extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* first bucket is [0, 1 + resolution)
* </pre>
*
* <code>double resolution = 1;</code>
* Generated from protobuf field <code>double resolution = 1;</code>
* @return float
*/
public function getResolution()
{
@ -52,24 +45,25 @@ class HistogramParams extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* first bucket is [0, 1 + resolution)
* </pre>
*
* <code>double resolution = 1;</code>
* Generated from protobuf field <code>double resolution = 1;</code>
* @param float $var
* @return $this
*/
public function setResolution($var)
{
GPBUtil::checkDouble($var);
$this->resolution = $var;
return $this;
}
/**
* <pre>
* use enough buckets to allow this value
* </pre>
*
* <code>double max_possible = 2;</code>
* Generated from protobuf field <code>double max_possible = 2;</code>
* @return float
*/
public function getMaxPossible()
{
@ -77,16 +71,18 @@ class HistogramParams extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* use enough buckets to allow this value
* </pre>
*
* <code>double max_possible = 2;</code>
* Generated from protobuf field <code>double max_possible = 2;</code>
* @param float $var
* @return $this
*/
public function setMaxPossible($var)
{
GPBUtil::checkDouble($var);
$this->max_possible = $var;
return $this;
}
}

@ -9,7 +9,7 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* Protobuf type <code>grpc.testing.LoadParams</code>
* Generated from protobuf message <code>grpc.testing.LoadParams</code>
*/
class LoadParams extends \Google\Protobuf\Internal\Message
{
@ -21,7 +21,8 @@ class LoadParams extends \Google\Protobuf\Internal\Message
}
/**
* <code>.grpc.testing.ClosedLoopParams closed_loop = 1;</code>
* Generated from protobuf field <code>.grpc.testing.ClosedLoopParams closed_loop = 1;</code>
* @return \Grpc\Testing\ClosedLoopParams
*/
public function getClosedLoop()
{
@ -29,16 +30,21 @@ class LoadParams extends \Google\Protobuf\Internal\Message
}
/**
* <code>.grpc.testing.ClosedLoopParams closed_loop = 1;</code>
* Generated from protobuf field <code>.grpc.testing.ClosedLoopParams closed_loop = 1;</code>
* @param \Grpc\Testing\ClosedLoopParams $var
* @return $this
*/
public function setClosedLoop(&$var)
public function setClosedLoop($var)
{
GPBUtil::checkMessage($var, \Grpc\Testing\ClosedLoopParams::class);
$this->writeOneof(1, $var);
return $this;
}
/**
* <code>.grpc.testing.PoissonParams poisson = 2;</code>
* Generated from protobuf field <code>.grpc.testing.PoissonParams poisson = 2;</code>
* @return \Grpc\Testing\PoissonParams
*/
public function getPoisson()
{
@ -46,14 +52,21 @@ class LoadParams extends \Google\Protobuf\Internal\Message
}
/**
* <code>.grpc.testing.PoissonParams poisson = 2;</code>
* Generated from protobuf field <code>.grpc.testing.PoissonParams poisson = 2;</code>
* @param \Grpc\Testing\PoissonParams $var
* @return $this
*/
public function setPoisson(&$var)
public function setPoisson($var)
{
GPBUtil::checkMessage($var, \Grpc\Testing\PoissonParams::class);
$this->writeOneof(2, $var);
return $this;
}
/**
* @return string
*/
public function getLoad()
{
return $this->whichOneof("load");

@ -9,20 +9,16 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* <pre>
* Request current stats
* </pre>
*
* Protobuf type <code>grpc.testing.Mark</code>
* Generated from protobuf message <code>grpc.testing.Mark</code>
*/
class Mark extends \Google\Protobuf\Internal\Message
{
/**
* <pre>
* if true, the stats will be reset after taking their snapshot.
* </pre>
*
* <code>bool reset = 1;</code>
* Generated from protobuf field <code>bool reset = 1;</code>
*/
private $reset = false;
@ -32,11 +28,10 @@ class Mark extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* if true, the stats will be reset after taking their snapshot.
* </pre>
*
* <code>bool reset = 1;</code>
* Generated from protobuf field <code>bool reset = 1;</code>
* @return bool
*/
public function getReset()
{
@ -44,16 +39,18 @@ class Mark extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* if true, the stats will be reset after taking their snapshot.
* </pre>
*
* <code>bool reset = 1;</code>
* Generated from protobuf field <code>bool reset = 1;</code>
* @param bool $var
* @return $this
*/
public function setReset($var)
{
GPBUtil::checkBool($var);
$this->reset = $var;
return $this;
}
}

@ -9,29 +9,23 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* <pre>
* A block of data, to simply increase gRPC message size.
* </pre>
*
* Protobuf type <code>grpc.testing.Payload</code>
* Generated from protobuf message <code>grpc.testing.Payload</code>
*/
class Payload extends \Google\Protobuf\Internal\Message
{
/**
* <pre>
* DEPRECATED, don't use. To be removed shortly.
* The type of data in body.
* </pre>
*
* <code>.grpc.testing.PayloadType type = 1;</code>
* Generated from protobuf field <code>.grpc.testing.PayloadType type = 1;</code>
*/
private $type = 0;
/**
* <pre>
* Primary contents of payload.
* </pre>
*
* <code>bytes body = 2;</code>
* Generated from protobuf field <code>bytes body = 2;</code>
*/
private $body = '';
@ -41,12 +35,11 @@ class Payload extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* DEPRECATED, don't use. To be removed shortly.
* The type of data in body.
* </pre>
*
* <code>.grpc.testing.PayloadType type = 1;</code>
* Generated from protobuf field <code>.grpc.testing.PayloadType type = 1;</code>
* @return int
*/
public function getType()
{
@ -54,25 +47,26 @@ class Payload extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* DEPRECATED, don't use. To be removed shortly.
* The type of data in body.
* </pre>
*
* <code>.grpc.testing.PayloadType type = 1;</code>
* Generated from protobuf field <code>.grpc.testing.PayloadType type = 1;</code>
* @param int $var
* @return $this
*/
public function setType($var)
{
GPBUtil::checkEnum($var, \Grpc\Testing\PayloadType::class);
$this->type = $var;
return $this;
}
/**
* <pre>
* Primary contents of payload.
* </pre>
*
* <code>bytes body = 2;</code>
* Generated from protobuf field <code>bytes body = 2;</code>
* @return string
*/
public function getBody()
{
@ -80,16 +74,18 @@ class Payload extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Primary contents of payload.
* </pre>
*
* <code>bytes body = 2;</code>
* Generated from protobuf field <code>bytes body = 2;</code>
* @param string $var
* @return $this
*/
public function setBody($var)
{
GPBUtil::checkString($var, False);
$this->body = $var;
return $this;
}
}

@ -9,7 +9,7 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* Protobuf type <code>grpc.testing.PayloadConfig</code>
* Generated from protobuf message <code>grpc.testing.PayloadConfig</code>
*/
class PayloadConfig extends \Google\Protobuf\Internal\Message
{
@ -21,7 +21,8 @@ class PayloadConfig extends \Google\Protobuf\Internal\Message
}
/**
* <code>.grpc.testing.ByteBufferParams bytebuf_params = 1;</code>
* Generated from protobuf field <code>.grpc.testing.ByteBufferParams bytebuf_params = 1;</code>
* @return \Grpc\Testing\ByteBufferParams
*/
public function getBytebufParams()
{
@ -29,16 +30,21 @@ class PayloadConfig extends \Google\Protobuf\Internal\Message
}
/**
* <code>.grpc.testing.ByteBufferParams bytebuf_params = 1;</code>
* Generated from protobuf field <code>.grpc.testing.ByteBufferParams bytebuf_params = 1;</code>
* @param \Grpc\Testing\ByteBufferParams $var
* @return $this
*/
public function setBytebufParams(&$var)
public function setBytebufParams($var)
{
GPBUtil::checkMessage($var, \Grpc\Testing\ByteBufferParams::class);
$this->writeOneof(1, $var);
return $this;
}
/**
* <code>.grpc.testing.SimpleProtoParams simple_params = 2;</code>
* Generated from protobuf field <code>.grpc.testing.SimpleProtoParams simple_params = 2;</code>
* @return \Grpc\Testing\SimpleProtoParams
*/
public function getSimpleParams()
{
@ -46,16 +52,21 @@ class PayloadConfig extends \Google\Protobuf\Internal\Message
}
/**
* <code>.grpc.testing.SimpleProtoParams simple_params = 2;</code>
* Generated from protobuf field <code>.grpc.testing.SimpleProtoParams simple_params = 2;</code>
* @param \Grpc\Testing\SimpleProtoParams $var
* @return $this
*/
public function setSimpleParams(&$var)
public function setSimpleParams($var)
{
GPBUtil::checkMessage($var, \Grpc\Testing\SimpleProtoParams::class);
$this->writeOneof(2, $var);
return $this;
}
/**
* <code>.grpc.testing.ComplexProtoParams complex_params = 3;</code>
* Generated from protobuf field <code>.grpc.testing.ComplexProtoParams complex_params = 3;</code>
* @return \Grpc\Testing\ComplexProtoParams
*/
public function getComplexParams()
{
@ -63,14 +74,21 @@ class PayloadConfig extends \Google\Protobuf\Internal\Message
}
/**
* <code>.grpc.testing.ComplexProtoParams complex_params = 3;</code>
* Generated from protobuf field <code>.grpc.testing.ComplexProtoParams complex_params = 3;</code>
* @param \Grpc\Testing\ComplexProtoParams $var
* @return $this
*/
public function setComplexParams(&$var)
public function setComplexParams($var)
{
GPBUtil::checkMessage($var, \Grpc\Testing\ComplexProtoParams::class);
$this->writeOneof(3, $var);
return $this;
}
/**
* @return string
*/
public function getPayload()
{
return $this->whichOneof("payload");

@ -5,21 +5,17 @@
namespace Grpc\Testing;
/**
* <pre>
* DEPRECATED, don't use. To be removed shortly.
* The type of payload that should be returned.
* </pre>
*
* Protobuf enum <code>grpc.testing.PayloadType</code>
* Protobuf enum <code>Grpc\Testing\PayloadType</code>
*/
class PayloadType
{
/**
* <pre>
* Compressable text format.
* </pre>
*
* <code>COMPRESSABLE = 0;</code>
* Generated from protobuf enum <code>COMPRESSABLE = 0;</code>
*/
const COMPRESSABLE = 0;
}

@ -9,21 +9,17 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* <pre>
* Parameters of poisson process distribution, which is a good representation
* of activity coming in from independent identical stationary sources.
* </pre>
*
* Protobuf type <code>grpc.testing.PoissonParams</code>
* Generated from protobuf message <code>grpc.testing.PoissonParams</code>
*/
class PoissonParams extends \Google\Protobuf\Internal\Message
{
/**
* <pre>
* The rate of arrivals (a.k.a. lambda parameter of the exp distribution).
* </pre>
*
* <code>double offered_load = 1;</code>
* Generated from protobuf field <code>double offered_load = 1;</code>
*/
private $offered_load = 0.0;
@ -33,11 +29,10 @@ class PoissonParams extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* The rate of arrivals (a.k.a. lambda parameter of the exp distribution).
* </pre>
*
* <code>double offered_load = 1;</code>
* Generated from protobuf field <code>double offered_load = 1;</code>
* @return float
*/
public function getOfferedLoad()
{
@ -45,16 +40,18 @@ class PoissonParams extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* The rate of arrivals (a.k.a. lambda parameter of the exp distribution).
* </pre>
*
* <code>double offered_load = 1;</code>
* Generated from protobuf field <code>double offered_load = 1;</code>
* @param float $var
* @return $this
*/
public function setOfferedLoad($var)
{
GPBUtil::checkDouble($var);
$this->offered_load = $var;
return $this;
}
}

@ -16,17 +16,19 @@
// See the License for the specific language governing permissions and
// limitations under the License.
//
namespace Grpc\Testing {
namespace Grpc\Testing;
class ProxyClientServiceClient extends \Grpc\BaseStub {
/**
*/
class ProxyClientServiceClient extends \Grpc\BaseStub {
/**
* @param string $hostname hostname
* @param array $opts channel options
* @param Grpc\Channel $channel (optional) re-use channel object
* @param \Grpc\Channel $channel (optional) re-use channel object
*/
public function __construct($hostname, $opts, $channel = null) {
parent::__construct($hostname, $opts, $channel);
parent::__construct($hostname, $opts, $channel);
}
/**
@ -36,10 +38,10 @@ namespace Grpc\Testing {
*/
public function GetConfig(\Grpc\Testing\Void $argument,
$metadata = [], $options = []) {
return $this->_simpleRequest('/grpc.testing.ProxyClientService/GetConfig',
$argument,
['\Grpc\Testing\ClientConfig', 'decode'],
$metadata, $options);
return $this->_simpleRequest('/grpc.testing.ProxyClientService/GetConfig',
$argument,
['\Grpc\Testing\ClientConfig', 'decode'],
$metadata, $options);
}
/**
@ -47,11 +49,19 @@ namespace Grpc\Testing {
* @param array $options call options
*/
public function ReportTime($metadata = [], $options = []) {
return $this->_clientStreamRequest('/grpc.testing.ProxyClientService/ReportTime',
['\Grpc\Testing\Void','decode'],
$metadata, $options);
return $this->_clientStreamRequest('/grpc.testing.ProxyClientService/ReportTime',
['\Grpc\Testing\Void','decode'],
$metadata, $options);
}
}
/**
* @param array $metadata metadata
* @param array $options call options
*/
public function ReportHist($metadata = [], $options = []) {
return $this->_clientStreamRequest('/grpc.testing.ProxyClientService/ReportHist',
['\Grpc\Testing\Void','decode'],
$metadata, $options);
}
}

@ -9,12 +9,12 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* Protobuf type <code>grpc.testing.ProxyStat</code>
* Generated from protobuf message <code>grpc.testing.ProxyStat</code>
*/
class ProxyStat extends \Google\Protobuf\Internal\Message
{
/**
* <code>double latency = 1;</code>
* Generated from protobuf field <code>double latency = 1;</code>
*/
private $latency = 0.0;
@ -24,7 +24,8 @@ class ProxyStat extends \Google\Protobuf\Internal\Message
}
/**
* <code>double latency = 1;</code>
* Generated from protobuf field <code>double latency = 1;</code>
* @return float
*/
public function getLatency()
{
@ -32,12 +33,16 @@ class ProxyStat extends \Google\Protobuf\Internal\Message
}
/**
* <code>double latency = 1;</code>
* Generated from protobuf field <code>double latency = 1;</code>
* @param float $var
* @return $this
*/
public function setLatency($var)
{
GPBUtil::checkDouble($var);
$this->latency = $var;
return $this;
}
}

@ -9,22 +9,20 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* <pre>
* For reconnect interop test only.
* Server tells client whether its reconnects are following the spec and the
* reconnect backoffs it saw.
* </pre>
*
* Protobuf type <code>grpc.testing.ReconnectInfo</code>
* Generated from protobuf message <code>grpc.testing.ReconnectInfo</code>
*/
class ReconnectInfo extends \Google\Protobuf\Internal\Message
{
/**
* <code>bool passed = 1;</code>
* Generated from protobuf field <code>bool passed = 1;</code>
*/
private $passed = false;
/**
* <code>repeated int32 backoff_ms = 2;</code>
* Generated from protobuf field <code>repeated int32 backoff_ms = 2;</code>
*/
private $backoff_ms;
@ -34,7 +32,8 @@ class ReconnectInfo extends \Google\Protobuf\Internal\Message
}
/**
* <code>bool passed = 1;</code>
* Generated from protobuf field <code>bool passed = 1;</code>
* @return bool
*/
public function getPassed()
{
@ -42,16 +41,21 @@ class ReconnectInfo extends \Google\Protobuf\Internal\Message
}
/**
* <code>bool passed = 1;</code>
* Generated from protobuf field <code>bool passed = 1;</code>
* @param bool $var
* @return $this
*/
public function setPassed($var)
{
GPBUtil::checkBool($var);
$this->passed = $var;
return $this;
}
/**
* <code>repeated int32 backoff_ms = 2;</code>
* Generated from protobuf field <code>repeated int32 backoff_ms = 2;</code>
* @return \Google\Protobuf\Internal\RepeatedField
*/
public function getBackoffMs()
{
@ -59,12 +63,16 @@ class ReconnectInfo extends \Google\Protobuf\Internal\Message
}
/**
* <code>repeated int32 backoff_ms = 2;</code>
* Generated from protobuf field <code>repeated int32 backoff_ms = 2;</code>
* @param int[]|\Google\Protobuf\Internal\RepeatedField $var
* @return $this
*/
public function setBackoffMs(&$var)
public function setBackoffMs($var)
{
GPBUtil::checkRepeatedField($var, \Google\Protobuf\Internal\GPBType::INT32);
$this->backoff_ms = $var;
$arr = GPBUtil::checkRepeatedField($var, \Google\Protobuf\Internal\GPBType::INT32);
$this->backoff_ms = $arr;
return $this;
}
}

@ -9,17 +9,15 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* <pre>
* For reconnect interop test only.
* Client tells server what reconnection parameters it used.
* </pre>
*
* Protobuf type <code>grpc.testing.ReconnectParams</code>
* Generated from protobuf message <code>grpc.testing.ReconnectParams</code>
*/
class ReconnectParams extends \Google\Protobuf\Internal\Message
{
/**
* <code>int32 max_reconnect_backoff_ms = 1;</code>
* Generated from protobuf field <code>int32 max_reconnect_backoff_ms = 1;</code>
*/
private $max_reconnect_backoff_ms = 0;
@ -29,7 +27,8 @@ class ReconnectParams extends \Google\Protobuf\Internal\Message
}
/**
* <code>int32 max_reconnect_backoff_ms = 1;</code>
* Generated from protobuf field <code>int32 max_reconnect_backoff_ms = 1;</code>
* @return int
*/
public function getMaxReconnectBackoffMs()
{
@ -37,12 +36,16 @@ class ReconnectParams extends \Google\Protobuf\Internal\Message
}
/**
* <code>int32 max_reconnect_backoff_ms = 1;</code>
* Generated from protobuf field <code>int32 max_reconnect_backoff_ms = 1;</code>
* @param int $var
* @return $this
*/
public function setMaxReconnectBackoffMs($var)
{
GPBUtil::checkInt32($var);
$this->max_reconnect_backoff_ms = $var;
return $this;
}
}

@ -0,0 +1,50 @@
<?php
// GENERATED CODE -- DO NOT EDIT!
// Original file comments:
// Copyright 2015 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.
//
// An integration test service that covers all the method signature permutations
// of unary/streaming requests/responses.
namespace Grpc\Testing;
/**
*/
class ReportQpsScenarioServiceClient extends \Grpc\BaseStub {
/**
* @param string $hostname hostname
* @param array $opts channel options
* @param \Grpc\Channel $channel (optional) re-use channel object
*/
public function __construct($hostname, $opts, $channel = null) {
parent::__construct($hostname, $opts, $channel);
}
/**
* Report results of a QPS test benchmark scenario.
* @param \Grpc\Testing\ScenarioResult $argument input argument
* @param array $metadata metadata
* @param array $options call options
*/
public function ReportScenario(\Grpc\Testing\ScenarioResult $argument,
$metadata = [], $options = []) {
return $this->_simpleRequest('/grpc.testing.ReportQpsScenarioService/ReportScenario',
$argument,
['\Grpc\Testing\Void', 'decode'],
$metadata, $options);
}
}

@ -9,16 +9,16 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* Protobuf type <code>grpc.testing.RequestResultCount</code>
* Generated from protobuf message <code>grpc.testing.RequestResultCount</code>
*/
class RequestResultCount extends \Google\Protobuf\Internal\Message
{
/**
* <code>int32 status_code = 1;</code>
* Generated from protobuf field <code>int32 status_code = 1;</code>
*/
private $status_code = 0;
/**
* <code>int64 count = 2;</code>
* Generated from protobuf field <code>int64 count = 2;</code>
*/
private $count = 0;
@ -28,7 +28,8 @@ class RequestResultCount extends \Google\Protobuf\Internal\Message
}
/**
* <code>int32 status_code = 1;</code>
* Generated from protobuf field <code>int32 status_code = 1;</code>
* @return int
*/
public function getStatusCode()
{
@ -36,16 +37,21 @@ class RequestResultCount extends \Google\Protobuf\Internal\Message
}
/**
* <code>int32 status_code = 1;</code>
* Generated from protobuf field <code>int32 status_code = 1;</code>
* @param int $var
* @return $this
*/
public function setStatusCode($var)
{
GPBUtil::checkInt32($var);
$this->status_code = $var;
return $this;
}
/**
* <code>int64 count = 2;</code>
* Generated from protobuf field <code>int64 count = 2;</code>
* @return int|string
*/
public function getCount()
{
@ -53,12 +59,16 @@ class RequestResultCount extends \Google\Protobuf\Internal\Message
}
/**
* <code>int64 count = 2;</code>
* Generated from protobuf field <code>int64 count = 2;</code>
* @param int|string $var
* @return $this
*/
public function setCount($var)
{
GPBUtil::checkInt64($var);
$this->count = $var;
return $this;
}
}

@ -9,40 +9,32 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* <pre>
* Configuration for a particular response.
* </pre>
*
* Protobuf type <code>grpc.testing.ResponseParameters</code>
* Generated from protobuf message <code>grpc.testing.ResponseParameters</code>
*/
class ResponseParameters extends \Google\Protobuf\Internal\Message
{
/**
* <pre>
* Desired payload sizes in responses from the server.
* </pre>
*
* <code>int32 size = 1;</code>
* Generated from protobuf field <code>int32 size = 1;</code>
*/
private $size = 0;
/**
* <pre>
* Desired interval between consecutive responses in the response stream in
* microseconds.
* </pre>
*
* <code>int32 interval_us = 2;</code>
* Generated from protobuf field <code>int32 interval_us = 2;</code>
*/
private $interval_us = 0;
/**
* <pre>
* Whether to request the server to compress the response. This field is
* "nullable" in order to interoperate seamlessly with clients not able to
* implement the full compression tests by introspecting the call to verify
* the response's compression status.
* </pre>
*
* <code>.grpc.testing.BoolValue compressed = 3;</code>
* Generated from protobuf field <code>.grpc.testing.BoolValue compressed = 3;</code>
*/
private $compressed = null;
@ -52,11 +44,10 @@ class ResponseParameters extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Desired payload sizes in responses from the server.
* </pre>
*
* <code>int32 size = 1;</code>
* Generated from protobuf field <code>int32 size = 1;</code>
* @return int
*/
public function getSize()
{
@ -64,25 +55,26 @@ class ResponseParameters extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Desired payload sizes in responses from the server.
* </pre>
*
* <code>int32 size = 1;</code>
* Generated from protobuf field <code>int32 size = 1;</code>
* @param int $var
* @return $this
*/
public function setSize($var)
{
GPBUtil::checkInt32($var);
$this->size = $var;
return $this;
}
/**
* <pre>
* Desired interval between consecutive responses in the response stream in
* microseconds.
* </pre>
*
* <code>int32 interval_us = 2;</code>
* Generated from protobuf field <code>int32 interval_us = 2;</code>
* @return int
*/
public function getIntervalUs()
{
@ -90,28 +82,29 @@ class ResponseParameters extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Desired interval between consecutive responses in the response stream in
* microseconds.
* </pre>
*
* <code>int32 interval_us = 2;</code>
* Generated from protobuf field <code>int32 interval_us = 2;</code>
* @param int $var
* @return $this
*/
public function setIntervalUs($var)
{
GPBUtil::checkInt32($var);
$this->interval_us = $var;
return $this;
}
/**
* <pre>
* Whether to request the server to compress the response. This field is
* "nullable" in order to interoperate seamlessly with clients not able to
* implement the full compression tests by introspecting the call to verify
* the response's compression status.
* </pre>
*
* <code>.grpc.testing.BoolValue compressed = 3;</code>
* Generated from protobuf field <code>.grpc.testing.BoolValue compressed = 3;</code>
* @return \Grpc\Testing\BoolValue
*/
public function getCompressed()
{
@ -119,19 +112,21 @@ class ResponseParameters extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Whether to request the server to compress the response. This field is
* "nullable" in order to interoperate seamlessly with clients not able to
* implement the full compression tests by introspecting the call to verify
* the response's compression status.
* </pre>
*
* <code>.grpc.testing.BoolValue compressed = 3;</code>
* Generated from protobuf field <code>.grpc.testing.BoolValue compressed = 3;</code>
* @param \Grpc\Testing\BoolValue $var
* @return $this
*/
public function setCompressed(&$var)
public function setCompressed($var)
{
GPBUtil::checkMessage($var, \Grpc\Testing\BoolValue::class);
$this->compressed = $var;
return $this;
}
}

@ -5,17 +5,29 @@
namespace Grpc\Testing;
/**
* Protobuf enum <code>grpc.testing.RpcType</code>
* Protobuf enum <code>Grpc\Testing\RpcType</code>
*/
class RpcType
{
/**
* <code>UNARY = 0;</code>
* Generated from protobuf enum <code>UNARY = 0;</code>
*/
const UNARY = 0;
/**
* <code>STREAMING = 1;</code>
* Generated from protobuf enum <code>STREAMING = 1;</code>
*/
const STREAMING = 1;
/**
* Generated from protobuf enum <code>STREAMING_FROM_CLIENT = 2;</code>
*/
const STREAMING_FROM_CLIENT = 2;
/**
* Generated from protobuf enum <code>STREAMING_FROM_SERVER = 3;</code>
*/
const STREAMING_FROM_SERVER = 3;
/**
* Generated from protobuf enum <code>STREAMING_BOTH_WAYS = 4;</code>
*/
const STREAMING_BOTH_WAYS = 4;
}

@ -9,76 +9,58 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* <pre>
* A single performance scenario: input to qps_json_driver
* </pre>
*
* Protobuf type <code>grpc.testing.Scenario</code>
* Generated from protobuf message <code>grpc.testing.Scenario</code>
*/
class Scenario extends \Google\Protobuf\Internal\Message
{
/**
* <pre>
* Human readable name for this scenario
* </pre>
*
* <code>string name = 1;</code>
* Generated from protobuf field <code>string name = 1;</code>
*/
private $name = '';
/**
* <pre>
* Client configuration
* </pre>
*
* <code>.grpc.testing.ClientConfig client_config = 2;</code>
* Generated from protobuf field <code>.grpc.testing.ClientConfig client_config = 2;</code>
*/
private $client_config = null;
/**
* <pre>
* Number of clients to start for the test
* </pre>
*
* <code>int32 num_clients = 3;</code>
* Generated from protobuf field <code>int32 num_clients = 3;</code>
*/
private $num_clients = 0;
/**
* <pre>
* Server configuration
* </pre>
*
* <code>.grpc.testing.ServerConfig server_config = 4;</code>
* Generated from protobuf field <code>.grpc.testing.ServerConfig server_config = 4;</code>
*/
private $server_config = null;
/**
* <pre>
* Number of servers to start for the test
* </pre>
*
* <code>int32 num_servers = 5;</code>
* Generated from protobuf field <code>int32 num_servers = 5;</code>
*/
private $num_servers = 0;
/**
* <pre>
* Warmup period, in seconds
* </pre>
*
* <code>int32 warmup_seconds = 6;</code>
* Generated from protobuf field <code>int32 warmup_seconds = 6;</code>
*/
private $warmup_seconds = 0;
/**
* <pre>
* Benchmark time, in seconds
* </pre>
*
* <code>int32 benchmark_seconds = 7;</code>
* Generated from protobuf field <code>int32 benchmark_seconds = 7;</code>
*/
private $benchmark_seconds = 0;
/**
* <pre>
* Number of workers to spawn locally (usually zero)
* </pre>
*
* <code>int32 spawn_local_worker_count = 8;</code>
* Generated from protobuf field <code>int32 spawn_local_worker_count = 8;</code>
*/
private $spawn_local_worker_count = 0;
@ -88,11 +70,10 @@ class Scenario extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Human readable name for this scenario
* </pre>
*
* <code>string name = 1;</code>
* Generated from protobuf field <code>string name = 1;</code>
* @return string
*/
public function getName()
{
@ -100,24 +81,25 @@ class Scenario extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Human readable name for this scenario
* </pre>
*
* <code>string name = 1;</code>
* Generated from protobuf field <code>string name = 1;</code>
* @param string $var
* @return $this
*/
public function setName($var)
{
GPBUtil::checkString($var, True);
$this->name = $var;
return $this;
}
/**
* <pre>
* Client configuration
* </pre>
*
* <code>.grpc.testing.ClientConfig client_config = 2;</code>
* Generated from protobuf field <code>.grpc.testing.ClientConfig client_config = 2;</code>
* @return \Grpc\Testing\ClientConfig
*/
public function getClientConfig()
{
@ -125,24 +107,25 @@ class Scenario extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Client configuration
* </pre>
*
* <code>.grpc.testing.ClientConfig client_config = 2;</code>
* Generated from protobuf field <code>.grpc.testing.ClientConfig client_config = 2;</code>
* @param \Grpc\Testing\ClientConfig $var
* @return $this
*/
public function setClientConfig(&$var)
public function setClientConfig($var)
{
GPBUtil::checkMessage($var, \Grpc\Testing\ClientConfig::class);
$this->client_config = $var;
return $this;
}
/**
* <pre>
* Number of clients to start for the test
* </pre>
*
* <code>int32 num_clients = 3;</code>
* Generated from protobuf field <code>int32 num_clients = 3;</code>
* @return int
*/
public function getNumClients()
{
@ -150,24 +133,25 @@ class Scenario extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Number of clients to start for the test
* </pre>
*
* <code>int32 num_clients = 3;</code>
* Generated from protobuf field <code>int32 num_clients = 3;</code>
* @param int $var
* @return $this
*/
public function setNumClients($var)
{
GPBUtil::checkInt32($var);
$this->num_clients = $var;
return $this;
}
/**
* <pre>
* Server configuration
* </pre>
*
* <code>.grpc.testing.ServerConfig server_config = 4;</code>
* Generated from protobuf field <code>.grpc.testing.ServerConfig server_config = 4;</code>
* @return \Grpc\Testing\ServerConfig
*/
public function getServerConfig()
{
@ -175,24 +159,25 @@ class Scenario extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Server configuration
* </pre>
*
* <code>.grpc.testing.ServerConfig server_config = 4;</code>
* Generated from protobuf field <code>.grpc.testing.ServerConfig server_config = 4;</code>
* @param \Grpc\Testing\ServerConfig $var
* @return $this
*/
public function setServerConfig(&$var)
public function setServerConfig($var)
{
GPBUtil::checkMessage($var, \Grpc\Testing\ServerConfig::class);
$this->server_config = $var;
return $this;
}
/**
* <pre>
* Number of servers to start for the test
* </pre>
*
* <code>int32 num_servers = 5;</code>
* Generated from protobuf field <code>int32 num_servers = 5;</code>
* @return int
*/
public function getNumServers()
{
@ -200,24 +185,25 @@ class Scenario extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Number of servers to start for the test
* </pre>
*
* <code>int32 num_servers = 5;</code>
* Generated from protobuf field <code>int32 num_servers = 5;</code>
* @param int $var
* @return $this
*/
public function setNumServers($var)
{
GPBUtil::checkInt32($var);
$this->num_servers = $var;
return $this;
}
/**
* <pre>
* Warmup period, in seconds
* </pre>
*
* <code>int32 warmup_seconds = 6;</code>
* Generated from protobuf field <code>int32 warmup_seconds = 6;</code>
* @return int
*/
public function getWarmupSeconds()
{
@ -225,24 +211,25 @@ class Scenario extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Warmup period, in seconds
* </pre>
*
* <code>int32 warmup_seconds = 6;</code>
* Generated from protobuf field <code>int32 warmup_seconds = 6;</code>
* @param int $var
* @return $this
*/
public function setWarmupSeconds($var)
{
GPBUtil::checkInt32($var);
$this->warmup_seconds = $var;
return $this;
}
/**
* <pre>
* Benchmark time, in seconds
* </pre>
*
* <code>int32 benchmark_seconds = 7;</code>
* Generated from protobuf field <code>int32 benchmark_seconds = 7;</code>
* @return int
*/
public function getBenchmarkSeconds()
{
@ -250,24 +237,25 @@ class Scenario extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Benchmark time, in seconds
* </pre>
*
* <code>int32 benchmark_seconds = 7;</code>
* Generated from protobuf field <code>int32 benchmark_seconds = 7;</code>
* @param int $var
* @return $this
*/
public function setBenchmarkSeconds($var)
{
GPBUtil::checkInt32($var);
$this->benchmark_seconds = $var;
return $this;
}
/**
* <pre>
* Number of workers to spawn locally (usually zero)
* </pre>
*
* <code>int32 spawn_local_worker_count = 8;</code>
* Generated from protobuf field <code>int32 spawn_local_worker_count = 8;</code>
* @return int
*/
public function getSpawnLocalWorkerCount()
{
@ -275,16 +263,18 @@ class Scenario extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Number of workers to spawn locally (usually zero)
* </pre>
*
* <code>int32 spawn_local_worker_count = 8;</code>
* Generated from protobuf field <code>int32 spawn_local_worker_count = 8;</code>
* @param int $var
* @return $this
*/
public function setSpawnLocalWorkerCount($var)
{
GPBUtil::checkInt32($var);
$this->spawn_local_worker_count = $var;
return $this;
}
}

@ -9,80 +9,62 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* <pre>
* Results of a single benchmark scenario.
* </pre>
*
* Protobuf type <code>grpc.testing.ScenarioResult</code>
* Generated from protobuf message <code>grpc.testing.ScenarioResult</code>
*/
class ScenarioResult extends \Google\Protobuf\Internal\Message
{
/**
* <pre>
* Inputs used to run the scenario.
* </pre>
*
* <code>.grpc.testing.Scenario scenario = 1;</code>
* Generated from protobuf field <code>.grpc.testing.Scenario scenario = 1;</code>
*/
private $scenario = null;
/**
* <pre>
* Histograms from all clients merged into one histogram.
* </pre>
*
* <code>.grpc.testing.HistogramData latencies = 2;</code>
* Generated from protobuf field <code>.grpc.testing.HistogramData latencies = 2;</code>
*/
private $latencies = null;
/**
* <pre>
* Client stats for each client
* </pre>
*
* <code>repeated .grpc.testing.ClientStats client_stats = 3;</code>
* Generated from protobuf field <code>repeated .grpc.testing.ClientStats client_stats = 3;</code>
*/
private $client_stats;
/**
* <pre>
* Server stats for each server
* </pre>
*
* <code>repeated .grpc.testing.ServerStats server_stats = 4;</code>
* Generated from protobuf field <code>repeated .grpc.testing.ServerStats server_stats = 4;</code>
*/
private $server_stats;
/**
* <pre>
* Number of cores available to each server
* </pre>
*
* <code>repeated int32 server_cores = 5;</code>
* Generated from protobuf field <code>repeated int32 server_cores = 5;</code>
*/
private $server_cores;
/**
* <pre>
* An after-the-fact computed summary
* </pre>
*
* <code>.grpc.testing.ScenarioResultSummary summary = 6;</code>
* Generated from protobuf field <code>.grpc.testing.ScenarioResultSummary summary = 6;</code>
*/
private $summary = null;
/**
* <pre>
* Information on success or failure of each worker
* </pre>
*
* <code>repeated bool client_success = 7;</code>
* Generated from protobuf field <code>repeated bool client_success = 7;</code>
*/
private $client_success;
/**
* <code>repeated bool server_success = 8;</code>
* Generated from protobuf field <code>repeated bool server_success = 8;</code>
*/
private $server_success;
/**
* <pre>
* Number of failed requests (one row per status code seen)
* </pre>
*
* <code>repeated .grpc.testing.RequestResultCount request_results = 9;</code>
* Generated from protobuf field <code>repeated .grpc.testing.RequestResultCount request_results = 9;</code>
*/
private $request_results;
@ -92,11 +74,10 @@ class ScenarioResult extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Inputs used to run the scenario.
* </pre>
*
* <code>.grpc.testing.Scenario scenario = 1;</code>
* Generated from protobuf field <code>.grpc.testing.Scenario scenario = 1;</code>
* @return \Grpc\Testing\Scenario
*/
public function getScenario()
{
@ -104,24 +85,25 @@ class ScenarioResult extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Inputs used to run the scenario.
* </pre>
*
* <code>.grpc.testing.Scenario scenario = 1;</code>
* Generated from protobuf field <code>.grpc.testing.Scenario scenario = 1;</code>
* @param \Grpc\Testing\Scenario $var
* @return $this
*/
public function setScenario(&$var)
public function setScenario($var)
{
GPBUtil::checkMessage($var, \Grpc\Testing\Scenario::class);
$this->scenario = $var;
return $this;
}
/**
* <pre>
* Histograms from all clients merged into one histogram.
* </pre>
*
* <code>.grpc.testing.HistogramData latencies = 2;</code>
* Generated from protobuf field <code>.grpc.testing.HistogramData latencies = 2;</code>
* @return \Grpc\Testing\HistogramData
*/
public function getLatencies()
{
@ -129,24 +111,25 @@ class ScenarioResult extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Histograms from all clients merged into one histogram.
* </pre>
*
* <code>.grpc.testing.HistogramData latencies = 2;</code>
* Generated from protobuf field <code>.grpc.testing.HistogramData latencies = 2;</code>
* @param \Grpc\Testing\HistogramData $var
* @return $this
*/
public function setLatencies(&$var)
public function setLatencies($var)
{
GPBUtil::checkMessage($var, \Grpc\Testing\HistogramData::class);
$this->latencies = $var;
return $this;
}
/**
* <pre>
* Client stats for each client
* </pre>
*
* <code>repeated .grpc.testing.ClientStats client_stats = 3;</code>
* Generated from protobuf field <code>repeated .grpc.testing.ClientStats client_stats = 3;</code>
* @return \Google\Protobuf\Internal\RepeatedField
*/
public function getClientStats()
{
@ -154,24 +137,25 @@ class ScenarioResult extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Client stats for each client
* </pre>
*
* <code>repeated .grpc.testing.ClientStats client_stats = 3;</code>
* Generated from protobuf field <code>repeated .grpc.testing.ClientStats client_stats = 3;</code>
* @param \Grpc\Testing\ClientStats[]|\Google\Protobuf\Internal\RepeatedField $var
* @return $this
*/
public function setClientStats(&$var)
public function setClientStats($var)
{
GPBUtil::checkRepeatedField($var, \Google\Protobuf\Internal\GPBType::MESSAGE, \Grpc\Testing\ClientStats::class);
$this->client_stats = $var;
$arr = GPBUtil::checkRepeatedField($var, \Google\Protobuf\Internal\GPBType::MESSAGE, \Grpc\Testing\ClientStats::class);
$this->client_stats = $arr;
return $this;
}
/**
* <pre>
* Server stats for each server
* </pre>
*
* <code>repeated .grpc.testing.ServerStats server_stats = 4;</code>
* Generated from protobuf field <code>repeated .grpc.testing.ServerStats server_stats = 4;</code>
* @return \Google\Protobuf\Internal\RepeatedField
*/
public function getServerStats()
{
@ -179,24 +163,25 @@ class ScenarioResult extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Server stats for each server
* </pre>
*
* <code>repeated .grpc.testing.ServerStats server_stats = 4;</code>
* Generated from protobuf field <code>repeated .grpc.testing.ServerStats server_stats = 4;</code>
* @param \Grpc\Testing\ServerStats[]|\Google\Protobuf\Internal\RepeatedField $var
* @return $this
*/
public function setServerStats(&$var)
public function setServerStats($var)
{
GPBUtil::checkRepeatedField($var, \Google\Protobuf\Internal\GPBType::MESSAGE, \Grpc\Testing\ServerStats::class);
$this->server_stats = $var;
$arr = GPBUtil::checkRepeatedField($var, \Google\Protobuf\Internal\GPBType::MESSAGE, \Grpc\Testing\ServerStats::class);
$this->server_stats = $arr;
return $this;
}
/**
* <pre>
* Number of cores available to each server
* </pre>
*
* <code>repeated int32 server_cores = 5;</code>
* Generated from protobuf field <code>repeated int32 server_cores = 5;</code>
* @return \Google\Protobuf\Internal\RepeatedField
*/
public function getServerCores()
{
@ -204,24 +189,25 @@ class ScenarioResult extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Number of cores available to each server
* </pre>
*
* <code>repeated int32 server_cores = 5;</code>
* Generated from protobuf field <code>repeated int32 server_cores = 5;</code>
* @param int[]|\Google\Protobuf\Internal\RepeatedField $var
* @return $this
*/
public function setServerCores(&$var)
public function setServerCores($var)
{
GPBUtil::checkRepeatedField($var, \Google\Protobuf\Internal\GPBType::INT32);
$this->server_cores = $var;
$arr = GPBUtil::checkRepeatedField($var, \Google\Protobuf\Internal\GPBType::INT32);
$this->server_cores = $arr;
return $this;
}
/**
* <pre>
* An after-the-fact computed summary
* </pre>
*
* <code>.grpc.testing.ScenarioResultSummary summary = 6;</code>
* Generated from protobuf field <code>.grpc.testing.ScenarioResultSummary summary = 6;</code>
* @return \Grpc\Testing\ScenarioResultSummary
*/
public function getSummary()
{
@ -229,24 +215,25 @@ class ScenarioResult extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* An after-the-fact computed summary
* </pre>
*
* <code>.grpc.testing.ScenarioResultSummary summary = 6;</code>
* Generated from protobuf field <code>.grpc.testing.ScenarioResultSummary summary = 6;</code>
* @param \Grpc\Testing\ScenarioResultSummary $var
* @return $this
*/
public function setSummary(&$var)
public function setSummary($var)
{
GPBUtil::checkMessage($var, \Grpc\Testing\ScenarioResultSummary::class);
$this->summary = $var;
return $this;
}
/**
* <pre>
* Information on success or failure of each worker
* </pre>
*
* <code>repeated bool client_success = 7;</code>
* Generated from protobuf field <code>repeated bool client_success = 7;</code>
* @return \Google\Protobuf\Internal\RepeatedField
*/
public function getClientSuccess()
{
@ -254,20 +241,23 @@ class ScenarioResult extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Information on success or failure of each worker
* </pre>
*
* <code>repeated bool client_success = 7;</code>
* Generated from protobuf field <code>repeated bool client_success = 7;</code>
* @param bool[]|\Google\Protobuf\Internal\RepeatedField $var
* @return $this
*/
public function setClientSuccess(&$var)
public function setClientSuccess($var)
{
GPBUtil::checkRepeatedField($var, \Google\Protobuf\Internal\GPBType::BOOL);
$this->client_success = $var;
$arr = GPBUtil::checkRepeatedField($var, \Google\Protobuf\Internal\GPBType::BOOL);
$this->client_success = $arr;
return $this;
}
/**
* <code>repeated bool server_success = 8;</code>
* Generated from protobuf field <code>repeated bool server_success = 8;</code>
* @return \Google\Protobuf\Internal\RepeatedField
*/
public function getServerSuccess()
{
@ -275,20 +265,23 @@ class ScenarioResult extends \Google\Protobuf\Internal\Message
}
/**
* <code>repeated bool server_success = 8;</code>
* Generated from protobuf field <code>repeated bool server_success = 8;</code>
* @param bool[]|\Google\Protobuf\Internal\RepeatedField $var
* @return $this
*/
public function setServerSuccess(&$var)
public function setServerSuccess($var)
{
GPBUtil::checkRepeatedField($var, \Google\Protobuf\Internal\GPBType::BOOL);
$this->server_success = $var;
$arr = GPBUtil::checkRepeatedField($var, \Google\Protobuf\Internal\GPBType::BOOL);
$this->server_success = $arr;
return $this;
}
/**
* <pre>
* Number of failed requests (one row per status code seen)
* </pre>
*
* <code>repeated .grpc.testing.RequestResultCount request_results = 9;</code>
* Generated from protobuf field <code>repeated .grpc.testing.RequestResultCount request_results = 9;</code>
* @return \Google\Protobuf\Internal\RepeatedField
*/
public function getRequestResults()
{
@ -296,16 +289,18 @@ class ScenarioResult extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Number of failed requests (one row per status code seen)
* </pre>
*
* <code>repeated .grpc.testing.RequestResultCount request_results = 9;</code>
* Generated from protobuf field <code>repeated .grpc.testing.RequestResultCount request_results = 9;</code>
* @param \Grpc\Testing\RequestResultCount[]|\Google\Protobuf\Internal\RepeatedField $var
* @return $this
*/
public function setRequestResults(&$var)
public function setRequestResults($var)
{
GPBUtil::checkRepeatedField($var, \Google\Protobuf\Internal\GPBType::MESSAGE, \Grpc\Testing\RequestResultCount::class);
$this->request_results = $var;
$arr = GPBUtil::checkRepeatedField($var, \Google\Protobuf\Internal\GPBType::MESSAGE, \Grpc\Testing\RequestResultCount::class);
$this->request_results = $arr;
return $this;
}
}

@ -9,107 +9,107 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* <pre>
* Basic summary that can be computed from ClientStats and ServerStats
* once the scenario has finished.
* </pre>
*
* Protobuf type <code>grpc.testing.ScenarioResultSummary</code>
* Generated from protobuf message <code>grpc.testing.ScenarioResultSummary</code>
*/
class ScenarioResultSummary extends \Google\Protobuf\Internal\Message
{
/**
* <pre>
* Total number of operations per second over all clients.
* </pre>
*
* <code>double qps = 1;</code>
* Generated from protobuf field <code>double qps = 1;</code>
*/
private $qps = 0.0;
/**
* <pre>
* QPS per one server core.
* </pre>
*
* <code>double qps_per_server_core = 2;</code>
* Generated from protobuf field <code>double qps_per_server_core = 2;</code>
*/
private $qps_per_server_core = 0.0;
/**
* <pre>
* server load based on system_time (0.85 =&gt; 85%)
* </pre>
* server load based on system_time (0.85 => 85%)
*
* <code>double server_system_time = 3;</code>
* Generated from protobuf field <code>double server_system_time = 3;</code>
*/
private $server_system_time = 0.0;
/**
* <pre>
* server load based on user_time (0.85 =&gt; 85%)
* </pre>
* server load based on user_time (0.85 => 85%)
*
* <code>double server_user_time = 4;</code>
* Generated from protobuf field <code>double server_user_time = 4;</code>
*/
private $server_user_time = 0.0;
/**
* <pre>
* client load based on system_time (0.85 =&gt; 85%)
* </pre>
* client load based on system_time (0.85 => 85%)
*
* <code>double client_system_time = 5;</code>
* Generated from protobuf field <code>double client_system_time = 5;</code>
*/
private $client_system_time = 0.0;
/**
* <pre>
* client load based on user_time (0.85 =&gt; 85%)
* </pre>
* client load based on user_time (0.85 => 85%)
*
* <code>double client_user_time = 6;</code>
* Generated from protobuf field <code>double client_user_time = 6;</code>
*/
private $client_user_time = 0.0;
/**
* <pre>
* X% latency percentiles (in nanoseconds)
* </pre>
*
* <code>double latency_50 = 7;</code>
* Generated from protobuf field <code>double latency_50 = 7;</code>
*/
private $latency_50 = 0.0;
/**
* <code>double latency_90 = 8;</code>
* Generated from protobuf field <code>double latency_90 = 8;</code>
*/
private $latency_90 = 0.0;
/**
* <code>double latency_95 = 9;</code>
* Generated from protobuf field <code>double latency_95 = 9;</code>
*/
private $latency_95 = 0.0;
/**
* <code>double latency_99 = 10;</code>
* Generated from protobuf field <code>double latency_99 = 10;</code>
*/
private $latency_99 = 0.0;
/**
* <code>double latency_999 = 11;</code>
* Generated from protobuf field <code>double latency_999 = 11;</code>
*/
private $latency_999 = 0.0;
/**
* <pre>
* server cpu usage percentage
* </pre>
*
* <code>double server_cpu_usage = 12;</code>
* Generated from protobuf field <code>double server_cpu_usage = 12;</code>
*/
private $server_cpu_usage = 0.0;
/**
* <pre>
* Number of requests that succeeded/failed
* </pre>
*
* <code>double successful_requests_per_second = 13;</code>
* Generated from protobuf field <code>double successful_requests_per_second = 13;</code>
*/
private $successful_requests_per_second = 0.0;
/**
* <code>double failed_requests_per_second = 14;</code>
* Generated from protobuf field <code>double failed_requests_per_second = 14;</code>
*/
private $failed_requests_per_second = 0.0;
/**
* Number of polls called inside completion queue per request
*
* Generated from protobuf field <code>double client_polls_per_request = 15;</code>
*/
private $client_polls_per_request = 0.0;
/**
* Generated from protobuf field <code>double server_polls_per_request = 16;</code>
*/
private $server_polls_per_request = 0.0;
/**
* Queries per CPU-sec over all servers or clients
*
* Generated from protobuf field <code>double server_queries_per_cpu_sec = 17;</code>
*/
private $server_queries_per_cpu_sec = 0.0;
/**
* Generated from protobuf field <code>double client_queries_per_cpu_sec = 18;</code>
*/
private $client_queries_per_cpu_sec = 0.0;
public function __construct() {
\GPBMetadata\Src\Proto\Grpc\Testing\Control::initOnce();
@ -117,11 +117,10 @@ class ScenarioResultSummary extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Total number of operations per second over all clients.
* </pre>
*
* <code>double qps = 1;</code>
* Generated from protobuf field <code>double qps = 1;</code>
* @return float
*/
public function getQps()
{
@ -129,24 +128,25 @@ class ScenarioResultSummary extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Total number of operations per second over all clients.
* </pre>
*
* <code>double qps = 1;</code>
* Generated from protobuf field <code>double qps = 1;</code>
* @param float $var
* @return $this
*/
public function setQps($var)
{
GPBUtil::checkDouble($var);
$this->qps = $var;
return $this;
}
/**
* <pre>
* QPS per one server core.
* </pre>
*
* <code>double qps_per_server_core = 2;</code>
* Generated from protobuf field <code>double qps_per_server_core = 2;</code>
* @return float
*/
public function getQpsPerServerCore()
{
@ -154,24 +154,25 @@ class ScenarioResultSummary extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* QPS per one server core.
* </pre>
*
* <code>double qps_per_server_core = 2;</code>
* Generated from protobuf field <code>double qps_per_server_core = 2;</code>
* @param float $var
* @return $this
*/
public function setQpsPerServerCore($var)
{
GPBUtil::checkDouble($var);
$this->qps_per_server_core = $var;
return $this;
}
/**
* <pre>
* server load based on system_time (0.85 =&gt; 85%)
* </pre>
* server load based on system_time (0.85 => 85%)
*
* <code>double server_system_time = 3;</code>
* Generated from protobuf field <code>double server_system_time = 3;</code>
* @return float
*/
public function getServerSystemTime()
{
@ -179,24 +180,25 @@ class ScenarioResultSummary extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* server load based on system_time (0.85 =&gt; 85%)
* </pre>
* server load based on system_time (0.85 => 85%)
*
* <code>double server_system_time = 3;</code>
* Generated from protobuf field <code>double server_system_time = 3;</code>
* @param float $var
* @return $this
*/
public function setServerSystemTime($var)
{
GPBUtil::checkDouble($var);
$this->server_system_time = $var;
return $this;
}
/**
* <pre>
* server load based on user_time (0.85 =&gt; 85%)
* </pre>
* server load based on user_time (0.85 => 85%)
*
* <code>double server_user_time = 4;</code>
* Generated from protobuf field <code>double server_user_time = 4;</code>
* @return float
*/
public function getServerUserTime()
{
@ -204,24 +206,25 @@ class ScenarioResultSummary extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* server load based on user_time (0.85 =&gt; 85%)
* </pre>
* server load based on user_time (0.85 => 85%)
*
* <code>double server_user_time = 4;</code>
* Generated from protobuf field <code>double server_user_time = 4;</code>
* @param float $var
* @return $this
*/
public function setServerUserTime($var)
{
GPBUtil::checkDouble($var);
$this->server_user_time = $var;
return $this;
}
/**
* <pre>
* client load based on system_time (0.85 =&gt; 85%)
* </pre>
* client load based on system_time (0.85 => 85%)
*
* <code>double client_system_time = 5;</code>
* Generated from protobuf field <code>double client_system_time = 5;</code>
* @return float
*/
public function getClientSystemTime()
{
@ -229,24 +232,25 @@ class ScenarioResultSummary extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* client load based on system_time (0.85 =&gt; 85%)
* </pre>
* client load based on system_time (0.85 => 85%)
*
* <code>double client_system_time = 5;</code>
* Generated from protobuf field <code>double client_system_time = 5;</code>
* @param float $var
* @return $this
*/
public function setClientSystemTime($var)
{
GPBUtil::checkDouble($var);
$this->client_system_time = $var;
return $this;
}
/**
* <pre>
* client load based on user_time (0.85 =&gt; 85%)
* </pre>
* client load based on user_time (0.85 => 85%)
*
* <code>double client_user_time = 6;</code>
* Generated from protobuf field <code>double client_user_time = 6;</code>
* @return float
*/
public function getClientUserTime()
{
@ -254,24 +258,25 @@ class ScenarioResultSummary extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* client load based on user_time (0.85 =&gt; 85%)
* </pre>
* client load based on user_time (0.85 => 85%)
*
* <code>double client_user_time = 6;</code>
* Generated from protobuf field <code>double client_user_time = 6;</code>
* @param float $var
* @return $this
*/
public function setClientUserTime($var)
{
GPBUtil::checkDouble($var);
$this->client_user_time = $var;
return $this;
}
/**
* <pre>
* X% latency percentiles (in nanoseconds)
* </pre>
*
* <code>double latency_50 = 7;</code>
* Generated from protobuf field <code>double latency_50 = 7;</code>
* @return float
*/
public function getLatency50()
{
@ -279,20 +284,23 @@ class ScenarioResultSummary extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* X% latency percentiles (in nanoseconds)
* </pre>
*
* <code>double latency_50 = 7;</code>
* Generated from protobuf field <code>double latency_50 = 7;</code>
* @param float $var
* @return $this
*/
public function setLatency50($var)
{
GPBUtil::checkDouble($var);
$this->latency_50 = $var;
return $this;
}
/**
* <code>double latency_90 = 8;</code>
* Generated from protobuf field <code>double latency_90 = 8;</code>
* @return float
*/
public function getLatency90()
{
@ -300,16 +308,21 @@ class ScenarioResultSummary extends \Google\Protobuf\Internal\Message
}
/**
* <code>double latency_90 = 8;</code>
* Generated from protobuf field <code>double latency_90 = 8;</code>
* @param float $var
* @return $this
*/
public function setLatency90($var)
{
GPBUtil::checkDouble($var);
$this->latency_90 = $var;
return $this;
}
/**
* <code>double latency_95 = 9;</code>
* Generated from protobuf field <code>double latency_95 = 9;</code>
* @return float
*/
public function getLatency95()
{
@ -317,16 +330,21 @@ class ScenarioResultSummary extends \Google\Protobuf\Internal\Message
}
/**
* <code>double latency_95 = 9;</code>
* Generated from protobuf field <code>double latency_95 = 9;</code>
* @param float $var
* @return $this
*/
public function setLatency95($var)
{
GPBUtil::checkDouble($var);
$this->latency_95 = $var;
return $this;
}
/**
* <code>double latency_99 = 10;</code>
* Generated from protobuf field <code>double latency_99 = 10;</code>
* @return float
*/
public function getLatency99()
{
@ -334,16 +352,21 @@ class ScenarioResultSummary extends \Google\Protobuf\Internal\Message
}
/**
* <code>double latency_99 = 10;</code>
* Generated from protobuf field <code>double latency_99 = 10;</code>
* @param float $var
* @return $this
*/
public function setLatency99($var)
{
GPBUtil::checkDouble($var);
$this->latency_99 = $var;
return $this;
}
/**
* <code>double latency_999 = 11;</code>
* Generated from protobuf field <code>double latency_999 = 11;</code>
* @return float
*/
public function getLatency999()
{
@ -351,20 +374,23 @@ class ScenarioResultSummary extends \Google\Protobuf\Internal\Message
}
/**
* <code>double latency_999 = 11;</code>
* Generated from protobuf field <code>double latency_999 = 11;</code>
* @param float $var
* @return $this
*/
public function setLatency999($var)
{
GPBUtil::checkDouble($var);
$this->latency_999 = $var;
return $this;
}
/**
* <pre>
* server cpu usage percentage
* </pre>
*
* <code>double server_cpu_usage = 12;</code>
* Generated from protobuf field <code>double server_cpu_usage = 12;</code>
* @return float
*/
public function getServerCpuUsage()
{
@ -372,24 +398,25 @@ class ScenarioResultSummary extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* server cpu usage percentage
* </pre>
*
* <code>double server_cpu_usage = 12;</code>
* Generated from protobuf field <code>double server_cpu_usage = 12;</code>
* @param float $var
* @return $this
*/
public function setServerCpuUsage($var)
{
GPBUtil::checkDouble($var);
$this->server_cpu_usage = $var;
return $this;
}
/**
* <pre>
* Number of requests that succeeded/failed
* </pre>
*
* <code>double successful_requests_per_second = 13;</code>
* Generated from protobuf field <code>double successful_requests_per_second = 13;</code>
* @return float
*/
public function getSuccessfulRequestsPerSecond()
{
@ -397,20 +424,23 @@ class ScenarioResultSummary extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Number of requests that succeeded/failed
* </pre>
*
* <code>double successful_requests_per_second = 13;</code>
* Generated from protobuf field <code>double successful_requests_per_second = 13;</code>
* @param float $var
* @return $this
*/
public function setSuccessfulRequestsPerSecond($var)
{
GPBUtil::checkDouble($var);
$this->successful_requests_per_second = $var;
return $this;
}
/**
* <code>double failed_requests_per_second = 14;</code>
* Generated from protobuf field <code>double failed_requests_per_second = 14;</code>
* @return float
*/
public function getFailedRequestsPerSecond()
{
@ -418,12 +448,112 @@ class ScenarioResultSummary extends \Google\Protobuf\Internal\Message
}
/**
* <code>double failed_requests_per_second = 14;</code>
* Generated from protobuf field <code>double failed_requests_per_second = 14;</code>
* @param float $var
* @return $this
*/
public function setFailedRequestsPerSecond($var)
{
GPBUtil::checkDouble($var);
$this->failed_requests_per_second = $var;
return $this;
}
/**
* Number of polls called inside completion queue per request
*
* Generated from protobuf field <code>double client_polls_per_request = 15;</code>
* @return float
*/
public function getClientPollsPerRequest()
{
return $this->client_polls_per_request;
}
/**
* Number of polls called inside completion queue per request
*
* Generated from protobuf field <code>double client_polls_per_request = 15;</code>
* @param float $var
* @return $this
*/
public function setClientPollsPerRequest($var)
{
GPBUtil::checkDouble($var);
$this->client_polls_per_request = $var;
return $this;
}
/**
* Generated from protobuf field <code>double server_polls_per_request = 16;</code>
* @return float
*/
public function getServerPollsPerRequest()
{
return $this->server_polls_per_request;
}
/**
* Generated from protobuf field <code>double server_polls_per_request = 16;</code>
* @param float $var
* @return $this
*/
public function setServerPollsPerRequest($var)
{
GPBUtil::checkDouble($var);
$this->server_polls_per_request = $var;
return $this;
}
/**
* Queries per CPU-sec over all servers or clients
*
* Generated from protobuf field <code>double server_queries_per_cpu_sec = 17;</code>
* @return float
*/
public function getServerQueriesPerCpuSec()
{
return $this->server_queries_per_cpu_sec;
}
/**
* Queries per CPU-sec over all servers or clients
*
* Generated from protobuf field <code>double server_queries_per_cpu_sec = 17;</code>
* @param float $var
* @return $this
*/
public function setServerQueriesPerCpuSec($var)
{
GPBUtil::checkDouble($var);
$this->server_queries_per_cpu_sec = $var;
return $this;
}
/**
* Generated from protobuf field <code>double client_queries_per_cpu_sec = 18;</code>
* @return float
*/
public function getClientQueriesPerCpuSec()
{
return $this->client_queries_per_cpu_sec;
}
/**
* Generated from protobuf field <code>double client_queries_per_cpu_sec = 18;</code>
* @param float $var
* @return $this
*/
public function setClientQueriesPerCpuSec($var)
{
GPBUtil::checkDouble($var);
$this->client_queries_per_cpu_sec = $var;
return $this;
}
}

@ -9,16 +9,14 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* <pre>
* A set of scenarios to be run with qps_json_driver
* </pre>
*
* Protobuf type <code>grpc.testing.Scenarios</code>
* Generated from protobuf message <code>grpc.testing.Scenarios</code>
*/
class Scenarios extends \Google\Protobuf\Internal\Message
{
/**
* <code>repeated .grpc.testing.Scenario scenarios = 1;</code>
* Generated from protobuf field <code>repeated .grpc.testing.Scenario scenarios = 1;</code>
*/
private $scenarios;
@ -28,7 +26,8 @@ class Scenarios extends \Google\Protobuf\Internal\Message
}
/**
* <code>repeated .grpc.testing.Scenario scenarios = 1;</code>
* Generated from protobuf field <code>repeated .grpc.testing.Scenario scenarios = 1;</code>
* @return \Google\Protobuf\Internal\RepeatedField
*/
public function getScenarios()
{
@ -36,12 +35,16 @@ class Scenarios extends \Google\Protobuf\Internal\Message
}
/**
* <code>repeated .grpc.testing.Scenario scenarios = 1;</code>
* Generated from protobuf field <code>repeated .grpc.testing.Scenario scenarios = 1;</code>
* @param \Grpc\Testing\Scenario[]|\Google\Protobuf\Internal\RepeatedField $var
* @return $this
*/
public function setScenarios(&$var)
public function setScenarios($var)
{
GPBUtil::checkRepeatedField($var, \Google\Protobuf\Internal\GPBType::MESSAGE, \Grpc\Testing\Scenario::class);
$this->scenarios = $var;
$arr = GPBUtil::checkRepeatedField($var, \Google\Protobuf\Internal\GPBType::MESSAGE, \Grpc\Testing\Scenario::class);
$this->scenarios = $arr;
return $this;
}
}

@ -9,22 +9,24 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* <pre>
* presence of SecurityParams implies use of TLS
* </pre>
*
* Protobuf type <code>grpc.testing.SecurityParams</code>
* Generated from protobuf message <code>grpc.testing.SecurityParams</code>
*/
class SecurityParams extends \Google\Protobuf\Internal\Message
{
/**
* <code>bool use_test_ca = 1;</code>
* Generated from protobuf field <code>bool use_test_ca = 1;</code>
*/
private $use_test_ca = false;
/**
* <code>string server_host_override = 2;</code>
* Generated from protobuf field <code>string server_host_override = 2;</code>
*/
private $server_host_override = '';
/**
* Generated from protobuf field <code>string cred_type = 3;</code>
*/
private $cred_type = '';
public function __construct() {
\GPBMetadata\Src\Proto\Grpc\Testing\Control::initOnce();
@ -32,7 +34,8 @@ class SecurityParams extends \Google\Protobuf\Internal\Message
}
/**
* <code>bool use_test_ca = 1;</code>
* Generated from protobuf field <code>bool use_test_ca = 1;</code>
* @return bool
*/
public function getUseTestCa()
{
@ -40,16 +43,21 @@ class SecurityParams extends \Google\Protobuf\Internal\Message
}
/**
* <code>bool use_test_ca = 1;</code>
* Generated from protobuf field <code>bool use_test_ca = 1;</code>
* @param bool $var
* @return $this
*/
public function setUseTestCa($var)
{
GPBUtil::checkBool($var);
$this->use_test_ca = $var;
return $this;
}
/**
* <code>string server_host_override = 2;</code>
* Generated from protobuf field <code>string server_host_override = 2;</code>
* @return string
*/
public function getServerHostOverride()
{
@ -57,12 +65,38 @@ class SecurityParams extends \Google\Protobuf\Internal\Message
}
/**
* <code>string server_host_override = 2;</code>
* Generated from protobuf field <code>string server_host_override = 2;</code>
* @param string $var
* @return $this
*/
public function setServerHostOverride($var)
{
GPBUtil::checkString($var, True);
$this->server_host_override = $var;
return $this;
}
/**
* Generated from protobuf field <code>string cred_type = 3;</code>
* @return string
*/
public function getCredType()
{
return $this->cred_type;
}
/**
* Generated from protobuf field <code>string cred_type = 3;</code>
* @param string $var
* @return $this
*/
public function setCredType($var)
{
GPBUtil::checkString($var, True);
$this->cred_type = $var;
return $this;
}
}

@ -9,7 +9,7 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* Protobuf type <code>grpc.testing.ServerArgs</code>
* Generated from protobuf message <code>grpc.testing.ServerArgs</code>
*/
class ServerArgs extends \Google\Protobuf\Internal\Message
{
@ -21,7 +21,8 @@ class ServerArgs extends \Google\Protobuf\Internal\Message
}
/**
* <code>.grpc.testing.ServerConfig setup = 1;</code>
* Generated from protobuf field <code>.grpc.testing.ServerConfig setup = 1;</code>
* @return \Grpc\Testing\ServerConfig
*/
public function getSetup()
{
@ -29,16 +30,21 @@ class ServerArgs extends \Google\Protobuf\Internal\Message
}
/**
* <code>.grpc.testing.ServerConfig setup = 1;</code>
* Generated from protobuf field <code>.grpc.testing.ServerConfig setup = 1;</code>
* @param \Grpc\Testing\ServerConfig $var
* @return $this
*/
public function setSetup(&$var)
public function setSetup($var)
{
GPBUtil::checkMessage($var, \Grpc\Testing\ServerConfig::class);
$this->writeOneof(1, $var);
return $this;
}
/**
* <code>.grpc.testing.Mark mark = 2;</code>
* Generated from protobuf field <code>.grpc.testing.Mark mark = 2;</code>
* @return \Grpc\Testing\Mark
*/
public function getMark()
{
@ -46,14 +52,21 @@ class ServerArgs extends \Google\Protobuf\Internal\Message
}
/**
* <code>.grpc.testing.Mark mark = 2;</code>
* Generated from protobuf field <code>.grpc.testing.Mark mark = 2;</code>
* @param \Grpc\Testing\Mark $var
* @return $this
*/
public function setMark(&$var)
public function setMark($var)
{
GPBUtil::checkMessage($var, \Grpc\Testing\Mark::class);
$this->writeOneof(2, $var);
return $this;
}
/**
* @return string
*/
public function getArgtype()
{
return $this->whichOneof("argtype");

@ -9,77 +9,73 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* Protobuf type <code>grpc.testing.ServerConfig</code>
* Generated from protobuf message <code>grpc.testing.ServerConfig</code>
*/
class ServerConfig extends \Google\Protobuf\Internal\Message
{
/**
* <code>.grpc.testing.ServerType server_type = 1;</code>
* Generated from protobuf field <code>.grpc.testing.ServerType server_type = 1;</code>
*/
private $server_type = 0;
/**
* <code>.grpc.testing.SecurityParams security_params = 2;</code>
* Generated from protobuf field <code>.grpc.testing.SecurityParams security_params = 2;</code>
*/
private $security_params = null;
/**
* <pre>
* Port on which to listen. Zero means pick unused port.
* </pre>
*
* <code>int32 port = 4;</code>
* Generated from protobuf field <code>int32 port = 4;</code>
*/
private $port = 0;
/**
* <pre>
* Only for async server. Number of threads used to serve the requests.
* </pre>
*
* <code>int32 async_server_threads = 7;</code>
* Generated from protobuf field <code>int32 async_server_threads = 7;</code>
*/
private $async_server_threads = 0;
/**
* <pre>
* Specify the number of cores to limit server to, if desired
* </pre>
*
* <code>int32 core_limit = 8;</code>
* Generated from protobuf field <code>int32 core_limit = 8;</code>
*/
private $core_limit = 0;
/**
* <pre>
* payload config, used in generic server.
* Note this must NOT be used in proto (non-generic) servers. For proto servers,
* 'response sizes' must be configured from the 'response_size' field of the
* 'SimpleRequest' objects in RPC requests.
* </pre>
*
* <code>.grpc.testing.PayloadConfig payload_config = 9;</code>
* Generated from protobuf field <code>.grpc.testing.PayloadConfig payload_config = 9;</code>
*/
private $payload_config = null;
/**
* <pre>
* Specify the cores we should run the server on, if desired
* </pre>
*
* <code>repeated int32 core_list = 10;</code>
* Generated from protobuf field <code>repeated int32 core_list = 10;</code>
*/
private $core_list;
/**
* <pre>
* If we use an OTHER_SERVER client_type, this string gives more detail
* </pre>
*
* <code>string other_server_api = 11;</code>
* Generated from protobuf field <code>string other_server_api = 11;</code>
*/
private $other_server_api = '';
/**
* <pre>
* Number of threads that share each completion queue
*
* Generated from protobuf field <code>int32 threads_per_cq = 12;</code>
*/
private $threads_per_cq = 0;
/**
* Buffer pool size (no buffer pool specified if unset)
* </pre>
*
* <code>int32 resource_quota_size = 1001;</code>
* Generated from protobuf field <code>int32 resource_quota_size = 1001;</code>
*/
private $resource_quota_size = 0;
/**
* Generated from protobuf field <code>repeated .grpc.testing.ChannelArg channel_args = 1002;</code>
*/
private $channel_args;
public function __construct() {
\GPBMetadata\Src\Proto\Grpc\Testing\Control::initOnce();
@ -87,7 +83,8 @@ class ServerConfig extends \Google\Protobuf\Internal\Message
}
/**
* <code>.grpc.testing.ServerType server_type = 1;</code>
* Generated from protobuf field <code>.grpc.testing.ServerType server_type = 1;</code>
* @return int
*/
public function getServerType()
{
@ -95,16 +92,21 @@ class ServerConfig extends \Google\Protobuf\Internal\Message
}
/**
* <code>.grpc.testing.ServerType server_type = 1;</code>
* Generated from protobuf field <code>.grpc.testing.ServerType server_type = 1;</code>
* @param int $var
* @return $this
*/
public function setServerType($var)
{
GPBUtil::checkEnum($var, \Grpc\Testing\ServerType::class);
$this->server_type = $var;
return $this;
}
/**
* <code>.grpc.testing.SecurityParams security_params = 2;</code>
* Generated from protobuf field <code>.grpc.testing.SecurityParams security_params = 2;</code>
* @return \Grpc\Testing\SecurityParams
*/
public function getSecurityParams()
{
@ -112,20 +114,23 @@ class ServerConfig extends \Google\Protobuf\Internal\Message
}
/**
* <code>.grpc.testing.SecurityParams security_params = 2;</code>
* Generated from protobuf field <code>.grpc.testing.SecurityParams security_params = 2;</code>
* @param \Grpc\Testing\SecurityParams $var
* @return $this
*/
public function setSecurityParams(&$var)
public function setSecurityParams($var)
{
GPBUtil::checkMessage($var, \Grpc\Testing\SecurityParams::class);
$this->security_params = $var;
return $this;
}
/**
* <pre>
* Port on which to listen. Zero means pick unused port.
* </pre>
*
* <code>int32 port = 4;</code>
* Generated from protobuf field <code>int32 port = 4;</code>
* @return int
*/
public function getPort()
{
@ -133,24 +138,25 @@ class ServerConfig extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Port on which to listen. Zero means pick unused port.
* </pre>
*
* <code>int32 port = 4;</code>
* Generated from protobuf field <code>int32 port = 4;</code>
* @param int $var
* @return $this
*/
public function setPort($var)
{
GPBUtil::checkInt32($var);
$this->port = $var;
return $this;
}
/**
* <pre>
* Only for async server. Number of threads used to serve the requests.
* </pre>
*
* <code>int32 async_server_threads = 7;</code>
* Generated from protobuf field <code>int32 async_server_threads = 7;</code>
* @return int
*/
public function getAsyncServerThreads()
{
@ -158,24 +164,25 @@ class ServerConfig extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Only for async server. Number of threads used to serve the requests.
* </pre>
*
* <code>int32 async_server_threads = 7;</code>
* Generated from protobuf field <code>int32 async_server_threads = 7;</code>
* @param int $var
* @return $this
*/
public function setAsyncServerThreads($var)
{
GPBUtil::checkInt32($var);
$this->async_server_threads = $var;
return $this;
}
/**
* <pre>
* Specify the number of cores to limit server to, if desired
* </pre>
*
* <code>int32 core_limit = 8;</code>
* Generated from protobuf field <code>int32 core_limit = 8;</code>
* @return int
*/
public function getCoreLimit()
{
@ -183,27 +190,28 @@ class ServerConfig extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Specify the number of cores to limit server to, if desired
* </pre>
*
* <code>int32 core_limit = 8;</code>
* Generated from protobuf field <code>int32 core_limit = 8;</code>
* @param int $var
* @return $this
*/
public function setCoreLimit($var)
{
GPBUtil::checkInt32($var);
$this->core_limit = $var;
return $this;
}
/**
* <pre>
* payload config, used in generic server.
* Note this must NOT be used in proto (non-generic) servers. For proto servers,
* 'response sizes' must be configured from the 'response_size' field of the
* 'SimpleRequest' objects in RPC requests.
* </pre>
*
* <code>.grpc.testing.PayloadConfig payload_config = 9;</code>
* Generated from protobuf field <code>.grpc.testing.PayloadConfig payload_config = 9;</code>
* @return \Grpc\Testing\PayloadConfig
*/
public function getPayloadConfig()
{
@ -211,27 +219,28 @@ class ServerConfig extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* payload config, used in generic server.
* Note this must NOT be used in proto (non-generic) servers. For proto servers,
* 'response sizes' must be configured from the 'response_size' field of the
* 'SimpleRequest' objects in RPC requests.
* </pre>
*
* <code>.grpc.testing.PayloadConfig payload_config = 9;</code>
* Generated from protobuf field <code>.grpc.testing.PayloadConfig payload_config = 9;</code>
* @param \Grpc\Testing\PayloadConfig $var
* @return $this
*/
public function setPayloadConfig(&$var)
public function setPayloadConfig($var)
{
GPBUtil::checkMessage($var, \Grpc\Testing\PayloadConfig::class);
$this->payload_config = $var;
return $this;
}
/**
* <pre>
* Specify the cores we should run the server on, if desired
* </pre>
*
* <code>repeated int32 core_list = 10;</code>
* Generated from protobuf field <code>repeated int32 core_list = 10;</code>
* @return \Google\Protobuf\Internal\RepeatedField
*/
public function getCoreList()
{
@ -239,24 +248,25 @@ class ServerConfig extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Specify the cores we should run the server on, if desired
* </pre>
*
* <code>repeated int32 core_list = 10;</code>
* Generated from protobuf field <code>repeated int32 core_list = 10;</code>
* @param int[]|\Google\Protobuf\Internal\RepeatedField $var
* @return $this
*/
public function setCoreList(&$var)
public function setCoreList($var)
{
GPBUtil::checkRepeatedField($var, \Google\Protobuf\Internal\GPBType::INT32);
$this->core_list = $var;
$arr = GPBUtil::checkRepeatedField($var, \Google\Protobuf\Internal\GPBType::INT32);
$this->core_list = $arr;
return $this;
}
/**
* <pre>
* If we use an OTHER_SERVER client_type, this string gives more detail
* </pre>
*
* <code>string other_server_api = 11;</code>
* Generated from protobuf field <code>string other_server_api = 11;</code>
* @return string
*/
public function getOtherServerApi()
{
@ -264,24 +274,51 @@ class ServerConfig extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* If we use an OTHER_SERVER client_type, this string gives more detail
* </pre>
*
* <code>string other_server_api = 11;</code>
* Generated from protobuf field <code>string other_server_api = 11;</code>
* @param string $var
* @return $this
*/
public function setOtherServerApi($var)
{
GPBUtil::checkString($var, True);
$this->other_server_api = $var;
return $this;
}
/**
* Number of threads that share each completion queue
*
* Generated from protobuf field <code>int32 threads_per_cq = 12;</code>
* @return int
*/
public function getThreadsPerCq()
{
return $this->threads_per_cq;
}
/**
* Number of threads that share each completion queue
*
* Generated from protobuf field <code>int32 threads_per_cq = 12;</code>
* @param int $var
* @return $this
*/
public function setThreadsPerCq($var)
{
GPBUtil::checkInt32($var);
$this->threads_per_cq = $var;
return $this;
}
/**
* <pre>
* Buffer pool size (no buffer pool specified if unset)
* </pre>
*
* <code>int32 resource_quota_size = 1001;</code>
* Generated from protobuf field <code>int32 resource_quota_size = 1001;</code>
* @return int
*/
public function getResourceQuotaSize()
{
@ -289,16 +326,40 @@ class ServerConfig extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Buffer pool size (no buffer pool specified if unset)
* </pre>
*
* <code>int32 resource_quota_size = 1001;</code>
* Generated from protobuf field <code>int32 resource_quota_size = 1001;</code>
* @param int $var
* @return $this
*/
public function setResourceQuotaSize($var)
{
GPBUtil::checkInt32($var);
$this->resource_quota_size = $var;
return $this;
}
/**
* Generated from protobuf field <code>repeated .grpc.testing.ChannelArg channel_args = 1002;</code>
* @return \Google\Protobuf\Internal\RepeatedField
*/
public function getChannelArgs()
{
return $this->channel_args;
}
/**
* Generated from protobuf field <code>repeated .grpc.testing.ChannelArg channel_args = 1002;</code>
* @param \Grpc\Testing\ChannelArg[]|\Google\Protobuf\Internal\RepeatedField $var
* @return $this
*/
public function setChannelArgs($var)
{
$arr = GPBUtil::checkRepeatedField($var, \Google\Protobuf\Internal\GPBType::MESSAGE, \Grpc\Testing\ChannelArg::class);
$this->channel_args = $arr;
return $this;
}
}

@ -9,51 +9,53 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* Protobuf type <code>grpc.testing.ServerStats</code>
* Generated from protobuf message <code>grpc.testing.ServerStats</code>
*/
class ServerStats extends \Google\Protobuf\Internal\Message
{
/**
* <pre>
* wall clock time change in seconds since last reset
* </pre>
*
* <code>double time_elapsed = 1;</code>
* Generated from protobuf field <code>double time_elapsed = 1;</code>
*/
private $time_elapsed = 0.0;
/**
* <pre>
* change in user time (in seconds) used by the server since last reset
* </pre>
*
* <code>double time_user = 2;</code>
* Generated from protobuf field <code>double time_user = 2;</code>
*/
private $time_user = 0.0;
/**
* <pre>
* change in server time (in seconds) used by the server process and all
* threads since last reset
* </pre>
*
* <code>double time_system = 3;</code>
* Generated from protobuf field <code>double time_system = 3;</code>
*/
private $time_system = 0.0;
/**
* <pre>
* change in total cpu time of the server (data from proc/stat)
* </pre>
*
* <code>uint64 total_cpu_time = 4;</code>
* Generated from protobuf field <code>uint64 total_cpu_time = 4;</code>
*/
private $total_cpu_time = 0;
/**
* <pre>
* change in idle time of the server (data from proc/stat)
* </pre>
*
* <code>uint64 idle_cpu_time = 5;</code>
* Generated from protobuf field <code>uint64 idle_cpu_time = 5;</code>
*/
private $idle_cpu_time = 0;
/**
* Number of polls called inside completion queue
*
* Generated from protobuf field <code>uint64 cq_poll_count = 6;</code>
*/
private $cq_poll_count = 0;
/**
* Core library stats
*
* Generated from protobuf field <code>.grpc.core.Stats core_stats = 7;</code>
*/
private $core_stats = null;
public function __construct() {
\GPBMetadata\Src\Proto\Grpc\Testing\Stats::initOnce();
@ -61,11 +63,10 @@ class ServerStats extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* wall clock time change in seconds since last reset
* </pre>
*
* <code>double time_elapsed = 1;</code>
* Generated from protobuf field <code>double time_elapsed = 1;</code>
* @return float
*/
public function getTimeElapsed()
{
@ -73,24 +74,25 @@ class ServerStats extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* wall clock time change in seconds since last reset
* </pre>
*
* <code>double time_elapsed = 1;</code>
* Generated from protobuf field <code>double time_elapsed = 1;</code>
* @param float $var
* @return $this
*/
public function setTimeElapsed($var)
{
GPBUtil::checkDouble($var);
$this->time_elapsed = $var;
return $this;
}
/**
* <pre>
* change in user time (in seconds) used by the server since last reset
* </pre>
*
* <code>double time_user = 2;</code>
* Generated from protobuf field <code>double time_user = 2;</code>
* @return float
*/
public function getTimeUser()
{
@ -98,25 +100,26 @@ class ServerStats extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* change in user time (in seconds) used by the server since last reset
* </pre>
*
* <code>double time_user = 2;</code>
* Generated from protobuf field <code>double time_user = 2;</code>
* @param float $var
* @return $this
*/
public function setTimeUser($var)
{
GPBUtil::checkDouble($var);
$this->time_user = $var;
return $this;
}
/**
* <pre>
* change in server time (in seconds) used by the server process and all
* threads since last reset
* </pre>
*
* <code>double time_system = 3;</code>
* Generated from protobuf field <code>double time_system = 3;</code>
* @return float
*/
public function getTimeSystem()
{
@ -124,25 +127,26 @@ class ServerStats extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* change in server time (in seconds) used by the server process and all
* threads since last reset
* </pre>
*
* <code>double time_system = 3;</code>
* Generated from protobuf field <code>double time_system = 3;</code>
* @param float $var
* @return $this
*/
public function setTimeSystem($var)
{
GPBUtil::checkDouble($var);
$this->time_system = $var;
return $this;
}
/**
* <pre>
* change in total cpu time of the server (data from proc/stat)
* </pre>
*
* <code>uint64 total_cpu_time = 4;</code>
* Generated from protobuf field <code>uint64 total_cpu_time = 4;</code>
* @return int|string
*/
public function getTotalCpuTime()
{
@ -150,24 +154,25 @@ class ServerStats extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* change in total cpu time of the server (data from proc/stat)
* </pre>
*
* <code>uint64 total_cpu_time = 4;</code>
* Generated from protobuf field <code>uint64 total_cpu_time = 4;</code>
* @param int|string $var
* @return $this
*/
public function setTotalCpuTime($var)
{
GPBUtil::checkUint64($var);
$this->total_cpu_time = $var;
return $this;
}
/**
* <pre>
* change in idle time of the server (data from proc/stat)
* </pre>
*
* <code>uint64 idle_cpu_time = 5;</code>
* Generated from protobuf field <code>uint64 idle_cpu_time = 5;</code>
* @return int|string
*/
public function getIdleCpuTime()
{
@ -175,16 +180,70 @@ class ServerStats extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* change in idle time of the server (data from proc/stat)
* </pre>
*
* <code>uint64 idle_cpu_time = 5;</code>
* Generated from protobuf field <code>uint64 idle_cpu_time = 5;</code>
* @param int|string $var
* @return $this
*/
public function setIdleCpuTime($var)
{
GPBUtil::checkUint64($var);
$this->idle_cpu_time = $var;
return $this;
}
/**
* Number of polls called inside completion queue
*
* Generated from protobuf field <code>uint64 cq_poll_count = 6;</code>
* @return int|string
*/
public function getCqPollCount()
{
return $this->cq_poll_count;
}
/**
* Number of polls called inside completion queue
*
* Generated from protobuf field <code>uint64 cq_poll_count = 6;</code>
* @param int|string $var
* @return $this
*/
public function setCqPollCount($var)
{
GPBUtil::checkUint64($var);
$this->cq_poll_count = $var;
return $this;
}
/**
* Core library stats
*
* Generated from protobuf field <code>.grpc.core.Stats core_stats = 7;</code>
* @return \Grpc\Core\Stats
*/
public function getCoreStats()
{
return $this->core_stats;
}
/**
* Core library stats
*
* Generated from protobuf field <code>.grpc.core.Stats core_stats = 7;</code>
* @param \Grpc\Core\Stats $var
* @return $this
*/
public function setCoreStats($var)
{
GPBUtil::checkMessage($var, \Grpc\Core\Stats::class);
$this->core_stats = $var;
return $this;
}
}

@ -9,28 +9,24 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* Protobuf type <code>grpc.testing.ServerStatus</code>
* Generated from protobuf message <code>grpc.testing.ServerStatus</code>
*/
class ServerStatus extends \Google\Protobuf\Internal\Message
{
/**
* <code>.grpc.testing.ServerStats stats = 1;</code>
* Generated from protobuf field <code>.grpc.testing.ServerStats stats = 1;</code>
*/
private $stats = null;
/**
* <pre>
* the port bound by the server
* </pre>
*
* <code>int32 port = 2;</code>
* Generated from protobuf field <code>int32 port = 2;</code>
*/
private $port = 0;
/**
* <pre>
* Number of cores available to the server
* </pre>
*
* <code>int32 cores = 3;</code>
* Generated from protobuf field <code>int32 cores = 3;</code>
*/
private $cores = 0;
@ -40,7 +36,8 @@ class ServerStatus extends \Google\Protobuf\Internal\Message
}
/**
* <code>.grpc.testing.ServerStats stats = 1;</code>
* Generated from protobuf field <code>.grpc.testing.ServerStats stats = 1;</code>
* @return \Grpc\Testing\ServerStats
*/
public function getStats()
{
@ -48,20 +45,23 @@ class ServerStatus extends \Google\Protobuf\Internal\Message
}
/**
* <code>.grpc.testing.ServerStats stats = 1;</code>
* Generated from protobuf field <code>.grpc.testing.ServerStats stats = 1;</code>
* @param \Grpc\Testing\ServerStats $var
* @return $this
*/
public function setStats(&$var)
public function setStats($var)
{
GPBUtil::checkMessage($var, \Grpc\Testing\ServerStats::class);
$this->stats = $var;
return $this;
}
/**
* <pre>
* the port bound by the server
* </pre>
*
* <code>int32 port = 2;</code>
* Generated from protobuf field <code>int32 port = 2;</code>
* @return int
*/
public function getPort()
{
@ -69,24 +69,25 @@ class ServerStatus extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* the port bound by the server
* </pre>
*
* <code>int32 port = 2;</code>
* Generated from protobuf field <code>int32 port = 2;</code>
* @param int $var
* @return $this
*/
public function setPort($var)
{
GPBUtil::checkInt32($var);
$this->port = $var;
return $this;
}
/**
* <pre>
* Number of cores available to the server
* </pre>
*
* <code>int32 cores = 3;</code>
* Generated from protobuf field <code>int32 cores = 3;</code>
* @return int
*/
public function getCores()
{
@ -94,16 +95,18 @@ class ServerStatus extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Number of cores available to the server
* </pre>
*
* <code>int32 cores = 3;</code>
* Generated from protobuf field <code>int32 cores = 3;</code>
* @param int $var
* @return $this
*/
public function setCores($var)
{
GPBUtil::checkInt32($var);
$this->cores = $var;
return $this;
}
}

@ -5,28 +5,26 @@
namespace Grpc\Testing;
/**
* Protobuf enum <code>grpc.testing.ServerType</code>
* Protobuf enum <code>Grpc\Testing\ServerType</code>
*/
class ServerType
{
/**
* <code>SYNC_SERVER = 0;</code>
* Generated from protobuf enum <code>SYNC_SERVER = 0;</code>
*/
const SYNC_SERVER = 0;
/**
* <code>ASYNC_SERVER = 1;</code>
* Generated from protobuf enum <code>ASYNC_SERVER = 1;</code>
*/
const ASYNC_SERVER = 1;
/**
* <code>ASYNC_GENERIC_SERVER = 2;</code>
* Generated from protobuf enum <code>ASYNC_GENERIC_SERVER = 2;</code>
*/
const ASYNC_GENERIC_SERVER = 2;
/**
* <pre>
* used for some language-specific variants
* </pre>
*
* <code>OTHER_SERVER = 3;</code>
* Generated from protobuf enum <code>OTHER_SERVER = 3;</code>
*/
const OTHER_SERVER = 3;
}

@ -9,16 +9,16 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* Protobuf type <code>grpc.testing.SimpleProtoParams</code>
* Generated from protobuf message <code>grpc.testing.SimpleProtoParams</code>
*/
class SimpleProtoParams extends \Google\Protobuf\Internal\Message
{
/**
* <code>int32 req_size = 1;</code>
* Generated from protobuf field <code>int32 req_size = 1;</code>
*/
private $req_size = 0;
/**
* <code>int32 resp_size = 2;</code>
* Generated from protobuf field <code>int32 resp_size = 2;</code>
*/
private $resp_size = 0;
@ -28,7 +28,8 @@ class SimpleProtoParams extends \Google\Protobuf\Internal\Message
}
/**
* <code>int32 req_size = 1;</code>
* Generated from protobuf field <code>int32 req_size = 1;</code>
* @return int
*/
public function getReqSize()
{
@ -36,16 +37,21 @@ class SimpleProtoParams extends \Google\Protobuf\Internal\Message
}
/**
* <code>int32 req_size = 1;</code>
* Generated from protobuf field <code>int32 req_size = 1;</code>
* @param int $var
* @return $this
*/
public function setReqSize($var)
{
GPBUtil::checkInt32($var);
$this->req_size = $var;
return $this;
}
/**
* <code>int32 resp_size = 2;</code>
* Generated from protobuf field <code>int32 resp_size = 2;</code>
* @return int
*/
public function getRespSize()
{
@ -53,12 +59,16 @@ class SimpleProtoParams extends \Google\Protobuf\Internal\Message
}
/**
* <code>int32 resp_size = 2;</code>
* Generated from protobuf field <code>int32 resp_size = 2;</code>
* @param int $var
* @return $this
*/
public function setRespSize($var)
{
GPBUtil::checkInt32($var);
$this->resp_size = $var;
return $this;
}
}

@ -9,81 +9,63 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* <pre>
* Unary request.
* </pre>
*
* Protobuf type <code>grpc.testing.SimpleRequest</code>
* Generated from protobuf message <code>grpc.testing.SimpleRequest</code>
*/
class SimpleRequest extends \Google\Protobuf\Internal\Message
{
/**
* <pre>
* DEPRECATED, don't use. To be removed shortly.
* Desired payload type in the response from the server.
* If response_type is RANDOM, server randomly chooses one from other formats.
* </pre>
*
* <code>.grpc.testing.PayloadType response_type = 1;</code>
* Generated from protobuf field <code>.grpc.testing.PayloadType response_type = 1;</code>
*/
private $response_type = 0;
/**
* <pre>
* Desired payload size in the response from the server.
* </pre>
*
* <code>int32 response_size = 2;</code>
* Generated from protobuf field <code>int32 response_size = 2;</code>
*/
private $response_size = 0;
/**
* <pre>
* Optional input payload sent along with the request.
* </pre>
*
* <code>.grpc.testing.Payload payload = 3;</code>
* Generated from protobuf field <code>.grpc.testing.Payload payload = 3;</code>
*/
private $payload = null;
/**
* <pre>
* Whether SimpleResponse should include username.
* </pre>
*
* <code>bool fill_username = 4;</code>
* Generated from protobuf field <code>bool fill_username = 4;</code>
*/
private $fill_username = false;
/**
* <pre>
* Whether SimpleResponse should include OAuth scope.
* </pre>
*
* <code>bool fill_oauth_scope = 5;</code>
* Generated from protobuf field <code>bool fill_oauth_scope = 5;</code>
*/
private $fill_oauth_scope = false;
/**
* <pre>
* Whether to request the server to compress the response. This field is
* "nullable" in order to interoperate seamlessly with clients not able to
* implement the full compression tests by introspecting the call to verify
* the response's compression status.
* </pre>
*
* <code>.grpc.testing.BoolValue response_compressed = 6;</code>
* Generated from protobuf field <code>.grpc.testing.BoolValue response_compressed = 6;</code>
*/
private $response_compressed = null;
/**
* <pre>
* Whether server should return a given status
* </pre>
*
* <code>.grpc.testing.EchoStatus response_status = 7;</code>
* Generated from protobuf field <code>.grpc.testing.EchoStatus response_status = 7;</code>
*/
private $response_status = null;
/**
* <pre>
* Whether the server should expect this request to be compressed.
* </pre>
*
* <code>.grpc.testing.BoolValue expect_compressed = 8;</code>
* Generated from protobuf field <code>.grpc.testing.BoolValue expect_compressed = 8;</code>
*/
private $expect_compressed = null;
@ -93,13 +75,12 @@ class SimpleRequest extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* DEPRECATED, don't use. To be removed shortly.
* Desired payload type in the response from the server.
* If response_type is RANDOM, server randomly chooses one from other formats.
* </pre>
*
* <code>.grpc.testing.PayloadType response_type = 1;</code>
* Generated from protobuf field <code>.grpc.testing.PayloadType response_type = 1;</code>
* @return int
*/
public function getResponseType()
{
@ -107,26 +88,27 @@ class SimpleRequest extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* DEPRECATED, don't use. To be removed shortly.
* Desired payload type in the response from the server.
* If response_type is RANDOM, server randomly chooses one from other formats.
* </pre>
*
* <code>.grpc.testing.PayloadType response_type = 1;</code>
* Generated from protobuf field <code>.grpc.testing.PayloadType response_type = 1;</code>
* @param int $var
* @return $this
*/
public function setResponseType($var)
{
GPBUtil::checkEnum($var, \Grpc\Testing\PayloadType::class);
$this->response_type = $var;
return $this;
}
/**
* <pre>
* Desired payload size in the response from the server.
* </pre>
*
* <code>int32 response_size = 2;</code>
* Generated from protobuf field <code>int32 response_size = 2;</code>
* @return int
*/
public function getResponseSize()
{
@ -134,24 +116,25 @@ class SimpleRequest extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Desired payload size in the response from the server.
* </pre>
*
* <code>int32 response_size = 2;</code>
* Generated from protobuf field <code>int32 response_size = 2;</code>
* @param int $var
* @return $this
*/
public function setResponseSize($var)
{
GPBUtil::checkInt32($var);
$this->response_size = $var;
return $this;
}
/**
* <pre>
* Optional input payload sent along with the request.
* </pre>
*
* <code>.grpc.testing.Payload payload = 3;</code>
* Generated from protobuf field <code>.grpc.testing.Payload payload = 3;</code>
* @return \Grpc\Testing\Payload
*/
public function getPayload()
{
@ -159,24 +142,25 @@ class SimpleRequest extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Optional input payload sent along with the request.
* </pre>
*
* <code>.grpc.testing.Payload payload = 3;</code>
* Generated from protobuf field <code>.grpc.testing.Payload payload = 3;</code>
* @param \Grpc\Testing\Payload $var
* @return $this
*/
public function setPayload(&$var)
public function setPayload($var)
{
GPBUtil::checkMessage($var, \Grpc\Testing\Payload::class);
$this->payload = $var;
return $this;
}
/**
* <pre>
* Whether SimpleResponse should include username.
* </pre>
*
* <code>bool fill_username = 4;</code>
* Generated from protobuf field <code>bool fill_username = 4;</code>
* @return bool
*/
public function getFillUsername()
{
@ -184,24 +168,25 @@ class SimpleRequest extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Whether SimpleResponse should include username.
* </pre>
*
* <code>bool fill_username = 4;</code>
* Generated from protobuf field <code>bool fill_username = 4;</code>
* @param bool $var
* @return $this
*/
public function setFillUsername($var)
{
GPBUtil::checkBool($var);
$this->fill_username = $var;
return $this;
}
/**
* <pre>
* Whether SimpleResponse should include OAuth scope.
* </pre>
*
* <code>bool fill_oauth_scope = 5;</code>
* Generated from protobuf field <code>bool fill_oauth_scope = 5;</code>
* @return bool
*/
public function getFillOauthScope()
{
@ -209,27 +194,28 @@ class SimpleRequest extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Whether SimpleResponse should include OAuth scope.
* </pre>
*
* <code>bool fill_oauth_scope = 5;</code>
* Generated from protobuf field <code>bool fill_oauth_scope = 5;</code>
* @param bool $var
* @return $this
*/
public function setFillOauthScope($var)
{
GPBUtil::checkBool($var);
$this->fill_oauth_scope = $var;
return $this;
}
/**
* <pre>
* Whether to request the server to compress the response. This field is
* "nullable" in order to interoperate seamlessly with clients not able to
* implement the full compression tests by introspecting the call to verify
* the response's compression status.
* </pre>
*
* <code>.grpc.testing.BoolValue response_compressed = 6;</code>
* Generated from protobuf field <code>.grpc.testing.BoolValue response_compressed = 6;</code>
* @return \Grpc\Testing\BoolValue
*/
public function getResponseCompressed()
{
@ -237,27 +223,28 @@ class SimpleRequest extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Whether to request the server to compress the response. This field is
* "nullable" in order to interoperate seamlessly with clients not able to
* implement the full compression tests by introspecting the call to verify
* the response's compression status.
* </pre>
*
* <code>.grpc.testing.BoolValue response_compressed = 6;</code>
* Generated from protobuf field <code>.grpc.testing.BoolValue response_compressed = 6;</code>
* @param \Grpc\Testing\BoolValue $var
* @return $this
*/
public function setResponseCompressed(&$var)
public function setResponseCompressed($var)
{
GPBUtil::checkMessage($var, \Grpc\Testing\BoolValue::class);
$this->response_compressed = $var;
return $this;
}
/**
* <pre>
* Whether server should return a given status
* </pre>
*
* <code>.grpc.testing.EchoStatus response_status = 7;</code>
* Generated from protobuf field <code>.grpc.testing.EchoStatus response_status = 7;</code>
* @return \Grpc\Testing\EchoStatus
*/
public function getResponseStatus()
{
@ -265,24 +252,25 @@ class SimpleRequest extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Whether server should return a given status
* </pre>
*
* <code>.grpc.testing.EchoStatus response_status = 7;</code>
* Generated from protobuf field <code>.grpc.testing.EchoStatus response_status = 7;</code>
* @param \Grpc\Testing\EchoStatus $var
* @return $this
*/
public function setResponseStatus(&$var)
public function setResponseStatus($var)
{
GPBUtil::checkMessage($var, \Grpc\Testing\EchoStatus::class);
$this->response_status = $var;
return $this;
}
/**
* <pre>
* Whether the server should expect this request to be compressed.
* </pre>
*
* <code>.grpc.testing.BoolValue expect_compressed = 8;</code>
* Generated from protobuf field <code>.grpc.testing.BoolValue expect_compressed = 8;</code>
* @return \Grpc\Testing\BoolValue
*/
public function getExpectCompressed()
{
@ -290,16 +278,18 @@ class SimpleRequest extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Whether the server should expect this request to be compressed.
* </pre>
*
* <code>.grpc.testing.BoolValue expect_compressed = 8;</code>
* Generated from protobuf field <code>.grpc.testing.BoolValue expect_compressed = 8;</code>
* @param \Grpc\Testing\BoolValue $var
* @return $this
*/
public function setExpectCompressed(&$var)
public function setExpectCompressed($var)
{
GPBUtil::checkMessage($var, \Grpc\Testing\BoolValue::class);
$this->expect_compressed = $var;
return $this;
}
}

@ -9,37 +9,29 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* <pre>
* Unary response, as configured by the request.
* </pre>
*
* Protobuf type <code>grpc.testing.SimpleResponse</code>
* Generated from protobuf message <code>grpc.testing.SimpleResponse</code>
*/
class SimpleResponse extends \Google\Protobuf\Internal\Message
{
/**
* <pre>
* Payload to increase message size.
* </pre>
*
* <code>.grpc.testing.Payload payload = 1;</code>
* Generated from protobuf field <code>.grpc.testing.Payload payload = 1;</code>
*/
private $payload = null;
/**
* <pre>
* The user the request came from, for verifying authentication was
* successful when the client expected it.
* </pre>
*
* <code>string username = 2;</code>
* Generated from protobuf field <code>string username = 2;</code>
*/
private $username = '';
/**
* <pre>
* OAuth scope.
* </pre>
*
* <code>string oauth_scope = 3;</code>
* Generated from protobuf field <code>string oauth_scope = 3;</code>
*/
private $oauth_scope = '';
@ -49,11 +41,10 @@ class SimpleResponse extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Payload to increase message size.
* </pre>
*
* <code>.grpc.testing.Payload payload = 1;</code>
* Generated from protobuf field <code>.grpc.testing.Payload payload = 1;</code>
* @return \Grpc\Testing\Payload
*/
public function getPayload()
{
@ -61,25 +52,26 @@ class SimpleResponse extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Payload to increase message size.
* </pre>
*
* <code>.grpc.testing.Payload payload = 1;</code>
* Generated from protobuf field <code>.grpc.testing.Payload payload = 1;</code>
* @param \Grpc\Testing\Payload $var
* @return $this
*/
public function setPayload(&$var)
public function setPayload($var)
{
GPBUtil::checkMessage($var, \Grpc\Testing\Payload::class);
$this->payload = $var;
return $this;
}
/**
* <pre>
* The user the request came from, for verifying authentication was
* successful when the client expected it.
* </pre>
*
* <code>string username = 2;</code>
* Generated from protobuf field <code>string username = 2;</code>
* @return string
*/
public function getUsername()
{
@ -87,25 +79,26 @@ class SimpleResponse extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* The user the request came from, for verifying authentication was
* successful when the client expected it.
* </pre>
*
* <code>string username = 2;</code>
* Generated from protobuf field <code>string username = 2;</code>
* @param string $var
* @return $this
*/
public function setUsername($var)
{
GPBUtil::checkString($var, True);
$this->username = $var;
return $this;
}
/**
* <pre>
* OAuth scope.
* </pre>
*
* <code>string oauth_scope = 3;</code>
* Generated from protobuf field <code>string oauth_scope = 3;</code>
* @return string
*/
public function getOauthScope()
{
@ -113,16 +106,18 @@ class SimpleResponse extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* OAuth scope.
* </pre>
*
* <code>string oauth_scope = 3;</code>
* Generated from protobuf field <code>string oauth_scope = 3;</code>
* @param string $var
* @return $this
*/
public function setOauthScope($var)
{
GPBUtil::checkString($var, True);
$this->oauth_scope = $var;
return $this;
}
}

@ -9,31 +9,25 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* <pre>
* Client-streaming request.
* </pre>
*
* Protobuf type <code>grpc.testing.StreamingInputCallRequest</code>
* Generated from protobuf message <code>grpc.testing.StreamingInputCallRequest</code>
*/
class StreamingInputCallRequest extends \Google\Protobuf\Internal\Message
{
/**
* <pre>
* Optional input payload sent along with the request.
* </pre>
*
* <code>.grpc.testing.Payload payload = 1;</code>
* Generated from protobuf field <code>.grpc.testing.Payload payload = 1;</code>
*/
private $payload = null;
/**
* <pre>
* Whether the server should expect this request to be compressed. This field
* is "nullable" in order to interoperate seamlessly with servers not able to
* implement the full compression tests by introspecting the call to verify
* the request's compression status.
* </pre>
*
* <code>.grpc.testing.BoolValue expect_compressed = 2;</code>
* Generated from protobuf field <code>.grpc.testing.BoolValue expect_compressed = 2;</code>
*/
private $expect_compressed = null;
@ -43,11 +37,10 @@ class StreamingInputCallRequest extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Optional input payload sent along with the request.
* </pre>
*
* <code>.grpc.testing.Payload payload = 1;</code>
* Generated from protobuf field <code>.grpc.testing.Payload payload = 1;</code>
* @return \Grpc\Testing\Payload
*/
public function getPayload()
{
@ -55,27 +48,28 @@ class StreamingInputCallRequest extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Optional input payload sent along with the request.
* </pre>
*
* <code>.grpc.testing.Payload payload = 1;</code>
* Generated from protobuf field <code>.grpc.testing.Payload payload = 1;</code>
* @param \Grpc\Testing\Payload $var
* @return $this
*/
public function setPayload(&$var)
public function setPayload($var)
{
GPBUtil::checkMessage($var, \Grpc\Testing\Payload::class);
$this->payload = $var;
return $this;
}
/**
* <pre>
* Whether the server should expect this request to be compressed. This field
* is "nullable" in order to interoperate seamlessly with servers not able to
* implement the full compression tests by introspecting the call to verify
* the request's compression status.
* </pre>
*
* <code>.grpc.testing.BoolValue expect_compressed = 2;</code>
* Generated from protobuf field <code>.grpc.testing.BoolValue expect_compressed = 2;</code>
* @return \Grpc\Testing\BoolValue
*/
public function getExpectCompressed()
{
@ -83,19 +77,21 @@ class StreamingInputCallRequest extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Whether the server should expect this request to be compressed. This field
* is "nullable" in order to interoperate seamlessly with servers not able to
* implement the full compression tests by introspecting the call to verify
* the request's compression status.
* </pre>
*
* <code>.grpc.testing.BoolValue expect_compressed = 2;</code>
* Generated from protobuf field <code>.grpc.testing.BoolValue expect_compressed = 2;</code>
* @param \Grpc\Testing\BoolValue $var
* @return $this
*/
public function setExpectCompressed(&$var)
public function setExpectCompressed($var)
{
GPBUtil::checkMessage($var, \Grpc\Testing\BoolValue::class);
$this->expect_compressed = $var;
return $this;
}
}

@ -9,20 +9,16 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* <pre>
* Client-streaming response.
* </pre>
*
* Protobuf type <code>grpc.testing.StreamingInputCallResponse</code>
* Generated from protobuf message <code>grpc.testing.StreamingInputCallResponse</code>
*/
class StreamingInputCallResponse extends \Google\Protobuf\Internal\Message
{
/**
* <pre>
* Aggregated size of payloads received from the client.
* </pre>
*
* <code>int32 aggregated_payload_size = 1;</code>
* Generated from protobuf field <code>int32 aggregated_payload_size = 1;</code>
*/
private $aggregated_payload_size = 0;
@ -32,11 +28,10 @@ class StreamingInputCallResponse extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Aggregated size of payloads received from the client.
* </pre>
*
* <code>int32 aggregated_payload_size = 1;</code>
* Generated from protobuf field <code>int32 aggregated_payload_size = 1;</code>
* @return int
*/
public function getAggregatedPayloadSize()
{
@ -44,16 +39,18 @@ class StreamingInputCallResponse extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Aggregated size of payloads received from the client.
* </pre>
*
* <code>int32 aggregated_payload_size = 1;</code>
* Generated from protobuf field <code>int32 aggregated_payload_size = 1;</code>
* @param int $var
* @return $this
*/
public function setAggregatedPayloadSize($var)
{
GPBUtil::checkInt32($var);
$this->aggregated_payload_size = $var;
return $this;
}
}

@ -9,48 +9,38 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* <pre>
* Server-streaming request.
* </pre>
*
* Protobuf type <code>grpc.testing.StreamingOutputCallRequest</code>
* Generated from protobuf message <code>grpc.testing.StreamingOutputCallRequest</code>
*/
class StreamingOutputCallRequest extends \Google\Protobuf\Internal\Message
{
/**
* <pre>
* DEPRECATED, don't use. To be removed shortly.
* Desired payload type in the response from the server.
* If response_type is RANDOM, the payload from each response in the stream
* might be of different types. This is to simulate a mixed type of payload
* stream.
* </pre>
*
* <code>.grpc.testing.PayloadType response_type = 1;</code>
* Generated from protobuf field <code>.grpc.testing.PayloadType response_type = 1;</code>
*/
private $response_type = 0;
/**
* <pre>
* Configuration for each expected response message.
* </pre>
*
* <code>repeated .grpc.testing.ResponseParameters response_parameters = 2;</code>
* Generated from protobuf field <code>repeated .grpc.testing.ResponseParameters response_parameters = 2;</code>
*/
private $response_parameters;
/**
* <pre>
* Optional input payload sent along with the request.
* </pre>
*
* <code>.grpc.testing.Payload payload = 3;</code>
* Generated from protobuf field <code>.grpc.testing.Payload payload = 3;</code>
*/
private $payload = null;
/**
* <pre>
* Whether server should return a given status
* </pre>
*
* <code>.grpc.testing.EchoStatus response_status = 7;</code>
* Generated from protobuf field <code>.grpc.testing.EchoStatus response_status = 7;</code>
*/
private $response_status = null;
@ -60,15 +50,14 @@ class StreamingOutputCallRequest extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* DEPRECATED, don't use. To be removed shortly.
* Desired payload type in the response from the server.
* If response_type is RANDOM, the payload from each response in the stream
* might be of different types. This is to simulate a mixed type of payload
* stream.
* </pre>
*
* <code>.grpc.testing.PayloadType response_type = 1;</code>
* Generated from protobuf field <code>.grpc.testing.PayloadType response_type = 1;</code>
* @return int
*/
public function getResponseType()
{
@ -76,28 +65,29 @@ class StreamingOutputCallRequest extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* DEPRECATED, don't use. To be removed shortly.
* Desired payload type in the response from the server.
* If response_type is RANDOM, the payload from each response in the stream
* might be of different types. This is to simulate a mixed type of payload
* stream.
* </pre>
*
* <code>.grpc.testing.PayloadType response_type = 1;</code>
* Generated from protobuf field <code>.grpc.testing.PayloadType response_type = 1;</code>
* @param int $var
* @return $this
*/
public function setResponseType($var)
{
GPBUtil::checkEnum($var, \Grpc\Testing\PayloadType::class);
$this->response_type = $var;
return $this;
}
/**
* <pre>
* Configuration for each expected response message.
* </pre>
*
* <code>repeated .grpc.testing.ResponseParameters response_parameters = 2;</code>
* Generated from protobuf field <code>repeated .grpc.testing.ResponseParameters response_parameters = 2;</code>
* @return \Google\Protobuf\Internal\RepeatedField
*/
public function getResponseParameters()
{
@ -105,24 +95,25 @@ class StreamingOutputCallRequest extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Configuration for each expected response message.
* </pre>
*
* <code>repeated .grpc.testing.ResponseParameters response_parameters = 2;</code>
* Generated from protobuf field <code>repeated .grpc.testing.ResponseParameters response_parameters = 2;</code>
* @param \Grpc\Testing\ResponseParameters[]|\Google\Protobuf\Internal\RepeatedField $var
* @return $this
*/
public function setResponseParameters(&$var)
public function setResponseParameters($var)
{
GPBUtil::checkRepeatedField($var, \Google\Protobuf\Internal\GPBType::MESSAGE, \Grpc\Testing\ResponseParameters::class);
$this->response_parameters = $var;
$arr = GPBUtil::checkRepeatedField($var, \Google\Protobuf\Internal\GPBType::MESSAGE, \Grpc\Testing\ResponseParameters::class);
$this->response_parameters = $arr;
return $this;
}
/**
* <pre>
* Optional input payload sent along with the request.
* </pre>
*
* <code>.grpc.testing.Payload payload = 3;</code>
* Generated from protobuf field <code>.grpc.testing.Payload payload = 3;</code>
* @return \Grpc\Testing\Payload
*/
public function getPayload()
{
@ -130,24 +121,25 @@ class StreamingOutputCallRequest extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Optional input payload sent along with the request.
* </pre>
*
* <code>.grpc.testing.Payload payload = 3;</code>
* Generated from protobuf field <code>.grpc.testing.Payload payload = 3;</code>
* @param \Grpc\Testing\Payload $var
* @return $this
*/
public function setPayload(&$var)
public function setPayload($var)
{
GPBUtil::checkMessage($var, \Grpc\Testing\Payload::class);
$this->payload = $var;
return $this;
}
/**
* <pre>
* Whether server should return a given status
* </pre>
*
* <code>.grpc.testing.EchoStatus response_status = 7;</code>
* Generated from protobuf field <code>.grpc.testing.EchoStatus response_status = 7;</code>
* @return \Grpc\Testing\EchoStatus
*/
public function getResponseStatus()
{
@ -155,16 +147,18 @@ class StreamingOutputCallRequest extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Whether server should return a given status
* </pre>
*
* <code>.grpc.testing.EchoStatus response_status = 7;</code>
* Generated from protobuf field <code>.grpc.testing.EchoStatus response_status = 7;</code>
* @param \Grpc\Testing\EchoStatus $var
* @return $this
*/
public function setResponseStatus(&$var)
public function setResponseStatus($var)
{
GPBUtil::checkMessage($var, \Grpc\Testing\EchoStatus::class);
$this->response_status = $var;
return $this;
}
}

@ -9,20 +9,16 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* <pre>
* Server-streaming response, as configured by the request and parameters.
* </pre>
*
* Protobuf type <code>grpc.testing.StreamingOutputCallResponse</code>
* Generated from protobuf message <code>grpc.testing.StreamingOutputCallResponse</code>
*/
class StreamingOutputCallResponse extends \Google\Protobuf\Internal\Message
{
/**
* <pre>
* Payload to increase response size.
* </pre>
*
* <code>.grpc.testing.Payload payload = 1;</code>
* Generated from protobuf field <code>.grpc.testing.Payload payload = 1;</code>
*/
private $payload = null;
@ -32,11 +28,10 @@ class StreamingOutputCallResponse extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Payload to increase response size.
* </pre>
*
* <code>.grpc.testing.Payload payload = 1;</code>
* Generated from protobuf field <code>.grpc.testing.Payload payload = 1;</code>
* @return \Grpc\Testing\Payload
*/
public function getPayload()
{
@ -44,16 +39,18 @@ class StreamingOutputCallResponse extends \Google\Protobuf\Internal\Message
}
/**
* <pre>
* Payload to increase response size.
* </pre>
*
* <code>.grpc.testing.Payload payload = 1;</code>
* Generated from protobuf field <code>.grpc.testing.Payload payload = 1;</code>
* @param \Grpc\Testing\Payload $var
* @return $this
*/
public function setPayload(&$var)
public function setPayload($var)
{
GPBUtil::checkMessage($var, \Grpc\Testing\Payload::class);
$this->payload = $var;
return $this;
}
}

@ -9,7 +9,7 @@ use Google\Protobuf\Internal\RepeatedField;
use Google\Protobuf\Internal\GPBUtil;
/**
* Protobuf type <code>grpc.testing.Void</code>
* Generated from protobuf message <code>grpc.testing.Void</code>
*/
class Void extends \Google\Protobuf\Internal\Message
{

@ -18,17 +18,19 @@
//
// An integration test service that covers all the method signature permutations
// of unary/streaming requests/responses.
namespace Grpc\Testing {
namespace Grpc\Testing;
class WorkerServiceClient extends \Grpc\BaseStub {
/**
*/
class WorkerServiceClient extends \Grpc\BaseStub {
/**
* @param string $hostname hostname
* @param array $opts channel options
* @param Grpc\Channel $channel (optional) re-use channel object
* @param \Grpc\Channel $channel (optional) re-use channel object
*/
public function __construct($hostname, $opts, $channel = null) {
parent::__construct($hostname, $opts, $channel);
parent::__construct($hostname, $opts, $channel);
}
/**
@ -42,9 +44,9 @@ namespace Grpc\Testing {
* @param array $options call options
*/
public function RunServer($metadata = [], $options = []) {
return $this->_bidiRequest('/grpc.testing.WorkerService/RunServer',
['\Grpc\Testing\ServerStatus','decode'],
$metadata, $options);
return $this->_bidiRequest('/grpc.testing.WorkerService/RunServer',
['\Grpc\Testing\ServerStatus','decode'],
$metadata, $options);
}
/**
@ -58,9 +60,9 @@ namespace Grpc\Testing {
* @param array $options call options
*/
public function RunClient($metadata = [], $options = []) {
return $this->_bidiRequest('/grpc.testing.WorkerService/RunClient',
['\Grpc\Testing\ClientStatus','decode'],
$metadata, $options);
return $this->_bidiRequest('/grpc.testing.WorkerService/RunClient',
['\Grpc\Testing\ClientStatus','decode'],
$metadata, $options);
}
/**
@ -71,10 +73,10 @@ namespace Grpc\Testing {
*/
public function CoreCount(\Grpc\Testing\CoreRequest $argument,
$metadata = [], $options = []) {
return $this->_simpleRequest('/grpc.testing.WorkerService/CoreCount',
$argument,
['\Grpc\Testing\CoreResponse', 'decode'],
$metadata, $options);
return $this->_simpleRequest('/grpc.testing.WorkerService/CoreCount',
$argument,
['\Grpc\Testing\CoreResponse', 'decode'],
$metadata, $options);
}
/**
@ -85,12 +87,10 @@ namespace Grpc\Testing {
*/
public function QuitWorker(\Grpc\Testing\Void $argument,
$metadata = [], $options = []) {
return $this->_simpleRequest('/grpc.testing.WorkerService/QuitWorker',
$argument,
['\Grpc\Testing\Void', 'decode'],
$metadata, $options);
return $this->_simpleRequest('/grpc.testing.WorkerService/QuitWorker',
$argument,
['\Grpc\Testing\Void', 'decode'],
$metadata, $options);
}
}
}

@ -0,0 +1,93 @@
<?php
/*
*
* Copyright 2017 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.
*
*/
// Histogram class for use in performance testing and measurement
class Histogram {
private $resolution;
private $max_possible;
private $sum;
private $sum_of_squares;
private $multiplier;
private $count;
private $min_seen;
private $max_seen;
private $buckets;
private function bucket_for($value) {
return (int)(log($value) / log($this->multiplier));
}
public function __construct($resolution, $max_possible) {
$this->resolution = $resolution;
$this->max_possible = $max_possible;
$this->sum = 0;
$this->sum_of_squares = 0;
$this->multiplier = 1+$resolution;
$this->count = 0;
$this->min_seen = $max_possible;
$this->max_seen = 0;
$this->buckets = array_fill(0, $this->bucket_for($max_possible)+1, 0);
}
public function add($value) {
$this->sum += $value;
$this->sum_of_squares += $value * $value;
$this->count += 1;
if ($value < $this->min_seen) {
$this->min_seen = $value;
}
if ($value > $this->max_seen) {
$this->max_seen = $value;
}
$this->buckets[$this->bucket_for($value)] += 1;
}
public function minimum() {
return $this->min_seen;
}
public function maximum() {
return $this->max_seen;
}
public function sum() {
return $this->sum;
}
public function sum_of_squares() {
return $this->sum_of_squares;
}
public function count() {
return $this->count;
}
public function contents() {
return $this->buckets;
}
public function clean() {
$this->sum = 0;
$this->sum_of_squares = 0;
$this->count = 0;
$this->min_seen = $this->max_possible;
$this->max_seen = 0;
$this->buckets = array_fill(0, $this->bucket_for($this->max_possible)+1, 0);
}
}

@ -15,6 +15,7 @@
syntax = "proto3";
import "src/proto/grpc/testing/control.proto";
import "src/proto/grpc/testing/stats.proto";
package grpc.testing;
@ -25,5 +26,6 @@ message ProxyStat {
service ProxyClientService {
rpc GetConfig(Void) returns (ClientConfig);
rpc ReportTime(stream ProxyStat) returns (Void);
rpc ReportHist(stream HistogramData) returns (Void);
}

@ -16,6 +16,8 @@
# Histogram class for use in performance testing and measurement
require 'thread'
class Histogram
# Determine the bucket index for a given value
# @param {number} value The value to check
@ -27,6 +29,7 @@ class Histogram
# @param {number} resolution The resolution of the histogram
# @param {number} max_possible The maximum value for the histogram
def initialize(resolution, max_possible)
@lock = Mutex.new
@resolution=resolution
@max_possible=max_possible
@sum=0
@ -70,4 +73,16 @@ class Histogram
def contents
@buckets
end
def merge(hist)
@lock.synchronize do
@min_seen = hist.min_seen
@max_seen = hist.max_seen
@sum += hist.sum
@sum_of_squares += hist.sum_of_squares
@count += hist.count
received_bucket = hist.bucket.to_a
@buckets = @buckets.map.with_index{ |m,i| m + received_bucket[i].to_i }
end
end
end

@ -41,32 +41,49 @@ class ProxyBenchmarkClientServiceImpl < Grpc::Testing::ProxyClientService::Servi
@histmax = config.histogram_params.max_possible
@histogram = Histogram.new(@histres, @histmax)
@start_time = Time.now
# TODO(vjpai): Support multiple client channels by spawning off a PHP client per channel
if @use_c_ext
puts "Use protobuf c extension"
command = "php -d extension=" + File.expand_path(File.dirname(__FILE__)) + "/../../php/tests/qps/vendor/google/protobuf/php/ext/google/protobuf/modules/protobuf.so " + "-d extension=" + File.expand_path(File.dirname(__FILE__)) + "/../../php/ext/grpc/modules/grpc.so " + File.expand_path(File.dirname(__FILE__)) + "/../../php/tests/qps/client.php " + @mytarget
else
puts "Use protobuf php extension"
command = "php -d extension=" + File.expand_path(File.dirname(__FILE__)) + "/../../php/ext/grpc/modules/grpc.so " + File.expand_path(File.dirname(__FILE__)) + "/../../php/tests/qps/client.php " + @mytarget
end
puts "Starting command: " + command
@php_pid = spawn(command)
@php_pid = Array.new(@config.client_channels)
(0..@config.client_channels-1).each do |chan|
Thread.new {
if @use_c_ext
puts "Use protobuf c extension"
command = "php -d extension=" + File.expand_path(File.dirname(__FILE__)) +
"/../../php/tests/qps/vendor/google/protobuf/php/ext/google/protobuf/modules/protobuf.so " +
"-d extension=" + File.expand_path(File.dirname(__FILE__)) + "/../../php/ext/grpc/modules/grpc.so " +
File.expand_path(File.dirname(__FILE__)) + "/../../php/tests/qps/client.php " + @mytarget + " #{chan%@config.server_targets.length}"
else
puts "Use protobuf php extension"
command = "php -d extension=" + File.expand_path(File.dirname(__FILE__)) + "/../../php/ext/grpc/modules/grpc.so " +
File.expand_path(File.dirname(__FILE__)) + "/../../php/tests/qps/client.php " + @mytarget + " #{chan%@config.server_targets.length}"
end
puts "[ruby proxy] Starting #{chan}th php-client command use c protobuf #{@use_c_ext}: " + command
@php_pid[chan] = spawn(command)
while true
sleep
end
}
end
end
def stop
Process.kill("TERM", @php_pid)
Process.wait(@php_pid)
(0..@config.client_channels-1).each do |chan|
Process.kill("TERM", @php_pid[chan])
Process.wait(@php_pid[chan])
end
end
def get_config(_args, _call)
puts "Answering get_config"
@config
end
def report_time(call)
puts "Starting a time reporting stream"
call.each_remote_read do |lat|
@histogram.add((lat.latency)*1e9)
end
Grpc::Testing::Void.new
end
def report_hist(call)
call.each_remote_read do |lat|
@histogram.merge(lat)
end
Grpc::Testing::Void.new
end
def mark(reset)
lat = Grpc::Testing::HistogramData.new(
bucket: @histogram.contents,
@ -135,7 +152,7 @@ def proxymain
opts.on('--driver_port PORT', '<port>') do |v|
options['driver_port'] = v
end
opts.on("-c", "--[no-]c_proto_ext", "Use protobuf C-extention") do |c|
opts.on("-c", "--[no-]use_protobuf_c_extension", "Use protobuf C-extention") do |c|
options[:c_ext] = c
end
end.parse!
@ -143,7 +160,8 @@ def proxymain
# Configure any errors with client or server child threads to surface
Thread.abort_on_exception = true
s = GRPC::RpcServer.new
# Make sure proxy_server can handle the large number of calls in benchmarks
s = GRPC::RpcServer.new(pool_size: 1024)
port = s.add_http2_port("0.0.0.0:" + options['driver_port'].to_s,
:this_port_is_insecure)
bmc = ProxyBenchmarkClientServiceImpl.new(port, options[:c_ext])

@ -4,6 +4,7 @@
require 'google/protobuf'
require 'src/proto/grpc/testing/control_pb'
require 'src/proto/grpc/testing/stats_pb'
Google::Protobuf::DescriptorPool.generated_pool.build do
add_message "grpc.testing.ProxyStat" do
optional :latency, :double, 1

@ -32,6 +32,7 @@ module Grpc
rpc :GetConfig, Void, ClientConfig
rpc :ReportTime, stream(ProxyStat), Void
rpc :ReportHist, stream(HistogramData), Void
end
Stub = Service.rpc_stub_class

@ -226,6 +226,7 @@ class Client {
}
virtual void DestroyMultithreading() = 0;
virtual void InitThreadFunc(size_t thread_idx) = 0;
virtual bool ThreadFunc(HistogramEntry* histogram, size_t thread_idx) = 0;
void SetupLoadTest(const ClientConfig& config, size_t num_threads) {
@ -299,13 +300,18 @@ class Client {
Thread& operator=(const Thread&);
void ThreadFunc() {
int wait_loop = 0;
while (!gpr_event_wait(
&client_->start_requests_,
gpr_time_add(gpr_now(GPR_CLOCK_REALTIME),
gpr_time_from_seconds(1, GPR_TIMESPAN)))) {
gpr_log(GPR_INFO, "Waiting for benchmark to start");
gpr_time_from_seconds(20, GPR_TIMESPAN)))) {
gpr_log(GPR_INFO, "%" PRIdPTR ": Waiting for benchmark to start (%d)",
idx_, wait_loop);
wait_loop++;
}
client_->InitThreadFunc(idx_);
for (;;) {
// run the loop body
HistogramEntry entry;
@ -380,6 +386,13 @@ class ClientImpl : public Client {
config.server_targets(i % config.server_targets_size()), config,
create_stub_, i);
}
std::vector<std::unique_ptr<std::thread>> connecting_threads;
for (auto& c : channels_) {
connecting_threads.emplace_back(c.WaitForReady());
}
for (auto& t : connecting_threads) {
t->join();
}
ClientRequestCreator<RequestType> create_req(&request_,
config.payload_config());
@ -414,14 +427,19 @@ class ClientImpl : public Client {
!config.security_params().use_test_ca(),
std::shared_ptr<CallCredentials>(), args);
gpr_log(GPR_INFO, "Connecting to %s", target.c_str());
GPR_ASSERT(channel_->WaitForConnected(
gpr_time_add(gpr_now(GPR_CLOCK_REALTIME),
gpr_time_from_seconds(300, GPR_TIMESPAN))));
stub_ = create_stub(channel_);
}
Channel* get_channel() { return channel_.get(); }
StubType* get_stub() { return stub_.get(); }
std::unique_ptr<std::thread> WaitForReady() {
return std::unique_ptr<std::thread>(new std::thread([this]() {
GPR_ASSERT(channel_->WaitForConnected(
gpr_time_add(gpr_now(GPR_CLOCK_REALTIME),
gpr_time_from_seconds(10, GPR_TIMESPAN))));
}));
}
private:
void set_channel_args(const ClientConfig& config, ChannelArguments* args) {
for (auto channel_arg : config.channel_args()) {

@ -236,6 +236,7 @@ class AsyncClient : public ClientImpl<StubType, RequestType> {
this->EndThreads(); // this needed for resolution
}
void InitThreadFunc(size_t thread_idx) override final {}
bool ThreadFunc(HistogramEntry* entry, size_t thread_idx) override final {
void* got_tag;
bool ok;

@ -103,6 +103,8 @@ class SynchronousUnaryClient final : public SynchronousClient {
}
~SynchronousUnaryClient() {}
void InitThreadFunc(size_t thread_idx) override {}
bool ThreadFunc(HistogramEntry* entry, size_t thread_idx) override {
if (!WaitToIssue(thread_idx)) {
return true;
@ -174,13 +176,7 @@ class SynchronousStreamingPingPongClient final
grpc::ClientReaderWriter<SimpleRequest, SimpleResponse>> {
public:
SynchronousStreamingPingPongClient(const ClientConfig& config)
: SynchronousStreamingClient(config) {
for (size_t thread_idx = 0; thread_idx < num_threads_; thread_idx++) {
auto* stub = channels_[thread_idx % channels_.size()].get_stub();
stream_[thread_idx] = stub->StreamingCall(&context_[thread_idx]);
messages_issued_[thread_idx] = 0;
}
}
: SynchronousStreamingClient(config) {}
~SynchronousStreamingPingPongClient() {
std::vector<std::thread> cleanup_threads;
for (size_t i = 0; i < num_threads_; i++) {
@ -196,6 +192,12 @@ class SynchronousStreamingPingPongClient final
}
}
void InitThreadFunc(size_t thread_idx) override {
auto* stub = channels_[thread_idx % channels_.size()].get_stub();
stream_[thread_idx] = stub->StreamingCall(&context_[thread_idx]);
messages_issued_[thread_idx] = 0;
}
bool ThreadFunc(HistogramEntry* entry, size_t thread_idx) override {
if (!WaitToIssue(thread_idx)) {
return true;
@ -228,14 +230,7 @@ class SynchronousStreamingFromClientClient final
: public SynchronousStreamingClient<grpc::ClientWriter<SimpleRequest>> {
public:
SynchronousStreamingFromClientClient(const ClientConfig& config)
: SynchronousStreamingClient(config), last_issue_(num_threads_) {
for (size_t thread_idx = 0; thread_idx < num_threads_; thread_idx++) {
auto* stub = channels_[thread_idx % channels_.size()].get_stub();
stream_[thread_idx] = stub->StreamingFromClient(&context_[thread_idx],
&responses_[thread_idx]);
last_issue_[thread_idx] = UsageTimer::Now();
}
}
: SynchronousStreamingClient(config), last_issue_(num_threads_) {}
~SynchronousStreamingFromClientClient() {
std::vector<std::thread> cleanup_threads;
for (size_t i = 0; i < num_threads_; i++) {
@ -251,6 +246,13 @@ class SynchronousStreamingFromClientClient final
}
}
void InitThreadFunc(size_t thread_idx) override {
auto* stub = channels_[thread_idx % channels_.size()].get_stub();
stream_[thread_idx] = stub->StreamingFromClient(&context_[thread_idx],
&responses_[thread_idx]);
last_issue_[thread_idx] = UsageTimer::Now();
}
bool ThreadFunc(HistogramEntry* entry, size_t thread_idx) override {
// Figure out how to make histogram sensible if this is rate-paced
if (!WaitToIssue(thread_idx)) {
@ -279,13 +281,12 @@ class SynchronousStreamingFromServerClient final
: public SynchronousStreamingClient<grpc::ClientReader<SimpleResponse>> {
public:
SynchronousStreamingFromServerClient(const ClientConfig& config)
: SynchronousStreamingClient(config), last_recv_(num_threads_) {
for (size_t thread_idx = 0; thread_idx < num_threads_; thread_idx++) {
auto* stub = channels_[thread_idx % channels_.size()].get_stub();
stream_[thread_idx] =
stub->StreamingFromServer(&context_[thread_idx], request_);
last_recv_[thread_idx] = UsageTimer::Now();
}
: SynchronousStreamingClient(config), last_recv_(num_threads_) {}
void InitThreadFunc(size_t thread_idx) override {
auto* stub = channels_[thread_idx % channels_.size()].get_stub();
stream_[thread_idx] =
stub->StreamingFromServer(&context_[thread_idx], request_);
last_recv_[thread_idx] = UsageTimer::Now();
}
bool ThreadFunc(HistogramEntry* entry, size_t thread_idx) override {
GPR_TIMER_SCOPE("SynchronousStreamingFromServerClient::ThreadFunc", 0);
@ -311,12 +312,7 @@ class SynchronousStreamingBothWaysClient final
grpc::ClientReaderWriter<SimpleRequest, SimpleResponse>> {
public:
SynchronousStreamingBothWaysClient(const ClientConfig& config)
: SynchronousStreamingClient(config) {
for (size_t thread_idx = 0; thread_idx < num_threads_; thread_idx++) {
auto* stub = channels_[thread_idx % channels_.size()].get_stub();
stream_[thread_idx] = stub->StreamingBothWays(&context_[thread_idx]);
}
}
: SynchronousStreamingClient(config) {}
~SynchronousStreamingBothWaysClient() {
std::vector<std::thread> cleanup_threads;
for (size_t i = 0; i < num_threads_; i++) {
@ -332,6 +328,10 @@ class SynchronousStreamingBothWaysClient final
}
}
void InitThreadFunc(size_t thread_idx) override {
auto* stub = channels_[thread_idx % channels_.size()].get_stub();
stream_[thread_idx] = stub->StreamingBothWays(&context_[thread_idx]);
}
bool ThreadFunc(HistogramEntry* entry, size_t thread_idx) override {
// TODO (vjpai): Do this
return true;

@ -40,11 +40,12 @@ pip install google-api-python-client --user python
export GOOGLE_APPLICATION_CREDENTIALS=${KOKORO_GFILE_DIR}/GrpcTesting-d0eeee2db331.json
# If this is a PR using RUN_TESTS_FLAGS var, then add flags to filter tests
if [ -n "$KOKORO_GITHUB_PULL_REQUEST_NUMBER" ] && [ -n "$RUN_TESTS_FLAGS" ]; then
brew install jq
ghprbTargetBranch=$(curl -s https://api.github.com/repos/grpc/grpc/pulls/$KOKORO_GITHUB_PULL_REQUEST_NUMBER | jq -r .base.ref)
export RUN_TESTS_FLAGS="$RUN_TESTS_FLAGS --filter_pr_tests --base_branch origin/$ghprbTargetBranch"
fi
# TODO(matt-kwong): enable after fixing brew issue
# if [ -n "$KOKORO_GITHUB_PULL_REQUEST_NUMBER" ] && [ -n "$RUN_TESTS_FLAGS" ]; then
# brew install jq
# ghprbTargetBranch=$(curl -s https://api.github.com/repos/grpc/grpc/pulls/$KOKORO_GITHUB_PULL_REQUEST_NUMBER | jq -r .base.ref)
# export RUN_TESTS_FLAGS="$RUN_TESTS_FLAGS --filter_pr_tests --base_branch origin/$ghprbTargetBranch"
# fi
set +ex # rvm script is very verbose and exits with errorcode
source $HOME/.rvm/scripts/rvm

@ -21,7 +21,7 @@ cd $(dirname $0)/../..
# run 8core client vs 8core server
tools/run_tests/run_performance_tests.py \
-l c++ csharp node ruby java python go node_express php_protobuf_php php_protobuf_c \
-l c++ csharp node ruby java python go node_express php7 php7_protobuf_c \
--netperf \
--category scalable \
--bq_result_table performance_test.performance_experiment \

@ -31,6 +31,7 @@ then
make CONFIG=${CONFIG} EMBED_OPENSSL=true EMBED_ZLIB=true qps_worker qps_json_driver -j8
fi
PHP_ALREADY_BUILT=""
for language in $@
do
case "$language" in
@ -43,6 +44,14 @@ do
"go")
tools/run_tests/performance/build_performance_go.sh
;;
"php7"|"php7_protobuf_c")
if [ -n "$PHP_ALREADY_BUILT" ]; then
echo "Skipping PHP build as already built by $PHP_ALREADY_BUILT"
else
PHP_ALREADY_BUILT=$language
tools/run_tests/performance/build_performance_php7.sh
fi
;;
"csharp")
python tools/run_tests/run_tests.py -l $language -c $CONFIG --build_only -j 8 --compiler coreclr
;;

@ -0,0 +1,30 @@
#!/bin/bash
# Copyright 2017 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)/../../..
CONFIG=${CONFIG:-opt}
python tools/run_tests/run_tests.py -l php7 -c $CONFIG --build_only -j 8
# Set up all dependences needed for PHP QPS test
cd src/php/tests/qps
composer install
# Install protobuf C-extension for php
cd vendor/google/protobuf/php/ext/google/protobuf
phpize
./configure
make

@ -17,17 +17,7 @@ source ~/.rvm/scripts/rvm
set -ex
cd $(dirname $0)/../../..
repo=$(pwd)
# First set up all dependences needed for PHP QPS test
cd $repo
cd src/php/tests/qps
composer install
# Install protobuf C-extension for php
cd vendor/google/protobuf/php/ext/google/protobuf
phpize
./configure
make
# The proxy worker for PHP is implemented in Ruby
cd $repo
ruby src/ruby/qps/proxy-worker.rb $@

@ -800,39 +800,54 @@ class RubyLanguage:
return 'ruby'
class PhpLanguage:
class Php7Language:
def __init__(self, use_protobuf_c_extension=False):
def __init__(self, php7_protobuf_c=False):
pass
self.use_protobuf_c_extension=use_protobuf_c_extension
self.php7_protobuf_c=php7_protobuf_c
self.safename = str(self)
def worker_cmdline(self):
if self.use_protobuf_c_extension:
return ['tools/run_tests/performance/run_worker_php.sh -c']
if self.php7_protobuf_c:
return ['tools/run_tests/performance/run_worker_php.sh --use_protobuf_c_extension']
return ['tools/run_tests/performance/run_worker_php.sh']
def worker_port_offset(self):
if self.php7_protobuf_c:
return 900
return 800
def scenarios(self):
php_extension_mode='php_protobuf_php_extension'
if self.use_protobuf_c_extension:
php_extension_mode='php_protobuf_c_extension'
php7_extension_mode='php7_protobuf_php_extension'
if self.php7_protobuf_c:
php7_extension_mode='php7_protobuf_c_extension'
yield _ping_pong_scenario(
'%s_to_cpp_protobuf_sync_unary_ping_pong' % php_extension_mode,
'%s_to_cpp_protobuf_sync_unary_ping_pong' % php7_extension_mode,
rpc_type='UNARY', client_type='SYNC_CLIENT', server_type='SYNC_SERVER',
server_language='c++', async_server_threads=1)
yield _ping_pong_scenario(
'%s_to_cpp_protobuf_sync_streaming_ping_pong' % php_extension_mode,
'%s_to_cpp_protobuf_sync_streaming_ping_pong' % php7_extension_mode,
rpc_type='STREAMING', client_type='SYNC_CLIENT', server_type='SYNC_SERVER',
server_language='c++', async_server_threads=1)
def __str__(self):
return 'php'
# TODO(ddyihai): Investigate why when async_server_threads=1/CPU usage 340%, the QPS performs
# better than async_server_threads=0/CPU usage 490%.
yield _ping_pong_scenario(
'%s_to_cpp_protobuf_sync_unary_qps_unconstrained' % php7_extension_mode,
rpc_type='UNARY', client_type='SYNC_CLIENT', server_type='ASYNC_SERVER',
server_language='c++', outstanding=1, async_server_threads=1, unconstrained_client='sync')
yield _ping_pong_scenario(
'%s_to_cpp_protobuf_sync_streaming_qps_unconstrained' % php7_extension_mode,
rpc_type='STREAMING', client_type='SYNC_CLIENT', server_type='ASYNC_SERVER',
server_language='c++', outstanding=1, async_server_threads=1, unconstrained_client='sync')
def __str__(self):
if self.php7_protobuf_c:
return 'php7_protobuf_c'
return 'php7'
class JavaLanguage:
@ -1031,8 +1046,8 @@ LANGUAGES = {
'node' : NodeLanguage(),
'node_express': NodeExpressLanguage(),
'ruby' : RubyLanguage(),
'php_protobuf_php' : PhpLanguage(),
'php_protobuf_c' : PhpLanguage(use_protobuf_c_extension=True),
'php7' : Php7Language(),
'php7_protobuf_c' : Php7Language(php7_protobuf_c=True),
'java' : JavaLanguage(),
'python' : PythonLanguage(),
'go' : GoLanguage(),

@ -412,7 +412,7 @@ class Jobset(object):
if current_cpu_cost + spec.cpu_cost <= self._maxjobs:
if len(self._running) < self._maxjobs_cpu_agnostic:
break
self.reap()
self.reap(spec.shortname, spec.cpu_cost)
if self.cancelled(): return False
job = Job(spec,
self._newline_on_success,
@ -424,7 +424,7 @@ class Jobset(object):
self.resultset[job.GetSpec().shortname] = []
return True
def reap(self):
def reap(self, waiting_for=None, waiting_for_cost=None):
"""Collect the dead jobs."""
while self._running:
dead = set()
@ -452,8 +452,12 @@ class Jobset(object):
sofar = now - self._start_time
remaining = sofar / self._completed * (self._remaining + len(self._running))
rstr = 'ETA %.1f sec; %s' % (remaining, rstr)
message('WAITING', '%s%d jobs running, %d complete, %d failed' % (
rstr, len(self._running), self._completed, self._failures))
if waiting_for is not None:
wstr = ' next: %s @ %.2f cpu' % (waiting_for, waiting_for_cost)
else:
wstr = ''
message('WAITING', '%s%d jobs running, %d complete, %d failed (load %.2f)%s' % (
rstr, len(self._running), self._completed, self._failures, self.cpu_cost(), wstr))
if platform_string() == 'windows':
time.sleep(0.1)
else:

Loading…
Cancel
Save