PHP: allow xDS interop client to start RPCs asynchronously (#25696)

* PHP: allow xDS interop client to start RPCs asynchronously

* Address review comments

* Remove adhoc test config
pull/25938/head
Stanley Cheung 4 years ago committed by GitHub
parent f3abcd3ace
commit c0a78774f8
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 63
      src/php/bin/run_xds_client.sh
  2. 100
      src/php/bin/xds_manager.py
  3. 4
      src/php/lib/Grpc/RpcServer.php
  4. 6
      src/php/tests/interop/Grpc/Testing/XdsUpdateClientConfigureServiceStub.php
  5. 356
      src/php/tests/interop/xds_client.php
  6. 54
      src/php/tests/interop/xds_empty_call.php
  7. 54
      src/php/tests/interop/xds_unary_call.php
  8. 17
      tools/internal_ci/linux/grpc_xds_php_test_in_docker.sh

@ -0,0 +1,63 @@
#!/bin/bash
# Copyright 2021 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.
# This script is being launched from the run_xds_tests.py runner and is
# responsible for managing child PHP processes.
cleanup () {
echo "Trapped SIGTERM. Cleaning up..."
set -x
kill -9 $PID1
kill -9 $PID2
running=false
set +x
}
trap cleanup SIGTERM
set -e
cd $(dirname $0)/../../..
root=$(pwd)
# tmp_file1 contains the list of RPCs (and their spec) the parent PHP
# process want executed
tmp_file1=$(mktemp)
# tmp_file2 contains the RPC result of each key initiated
tmp_file2=$(mktemp)
set -x
# This is the PHP parent process, who is primarily responding to the
# run_xds_tests.py runner's stats requests
php -d extension=grpc.so -d extension=pthreads.so \
src/php/tests/interop/xds_client.php $1 $2 $3 $4 $5 $6 \
--tmp_file1=$tmp_file1 --tmp_file2=$tmp_file2 &
PID1=$!
# This script watches RPCs written to tmp_file1, spawn off more PHP
# child processes to execute them, and writes the result to tmp_file2
python3 -u src/php/bin/xds_manager.py \
--tmp_file1=$tmp_file1 --tmp_file2=$tmp_file2 \
--bootstrap_path=$GRPC_XDS_BOOTSTRAP &
PID2=$!
set +x
# This will be killed by a SIGTERM signal from the run_xds_tests.py
# runner
running=true
while $running
do
sleep 1
done

@ -0,0 +1,100 @@
#!/usr/bin/env python
# Copyright 2021 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.
"""Manage PHP child processes for the main PHP xDS Interop client"""
import argparse
import fcntl
import os
import subprocess
# This script is being launched from src/php/bin/run_xds_client.sh
# to manage PHP child processes which will send 1 RPC each
# asynchronously. This script keeps track of all those open
# processes and reports back to the main PHP interop client each
# of the child RPCs' status code.
if __name__ == '__main__':
parser = argparse.ArgumentParser()
parser.add_argument('--tmp_file1', nargs='?', default='')
parser.add_argument('--tmp_file2', nargs='?', default='')
parser.add_argument('--bootstrap_path', nargs='?', default='')
args = parser.parse_args()
server_address = ''
rpcs_started = []
open_processes = {}
client_env = dict(os.environ)
client_env['GRPC_XDS_BOOTSTRAP'] = args.bootstrap_path
while True:
# tmp_file1 contains a list of RPCs (and their spec) the parent process
# wants executed
f1 = open(args.tmp_file1, 'r+')
fcntl.flock(f1, fcntl.LOCK_EX)
while True:
key = f1.readline()
if not key:
break
key = key.strip()
if key.startswith('server_address'):
if not server_address:
server_address = key[15:]
elif not key in rpcs_started:
# format here needs to be in sync with
# src/php/tests/interop/xds_client.php
items = key.split('|')
num = items[0]
rpc_behavior = items[2]
timeout_sec = items[3]
if items[1] == 'UnaryCall':
p = subprocess.Popen([
'php', '-d', 'extension=grpc.so', '-d',
'extension=pthreads.so',
'src/php/tests/interop/xds_unary_call.php',
'--server=' + server_address, '--num=' + str(num),
'--rpc_behavior=' + rpc_behavior,
'--timeout_sec=' + timeout_sec
],
env=client_env)
elif items[1] == 'EmptyCall':
p = subprocess.Popen([
'php', '-d', 'extension=grpc.so', '-d',
'extension=pthreads.so',
'src/php/tests/interop/xds_empty_call.php',
'--server=' + server_address, '--num=' + str(num),
'--rpc_behavior=' + rpc_behavior,
'--timeout=' + timeout_sec
],
env=client_env)
else:
continue
rpcs_started.append(key)
open_processes[key] = p
f1.truncate(0)
fcntl.flock(f1, fcntl.LOCK_UN)
f1.close()
# tmp_file2 contains the RPC result of each key received from tmp_file1
f2 = open(args.tmp_file2, 'a')
fcntl.flock(f2, fcntl.LOCK_EX)
keys_to_delete = []
for key, process in open_processes.items():
result = process.poll()
if result is not None:
# format here needs to be in sync with
# src/php/tests/interop/xds_client.php
f2.write(key + ',' + str(process.returncode) + "\n")
keys_to_delete.append(key)
for key in keys_to_delete:
del open_processes[key]
fcntl.flock(f2, fcntl.LOCK_UN)
f2.close()

@ -51,9 +51,7 @@ class RpcServer extends Server
$event = $this->call->startBatch([
OP_RECV_MESSAGE => true,
]);
if (!$event->message) {
throw new Exception("Did not receive a proper message");
}
$request->clear();
$request->mergeFromString($event->message);
return $request;
}

@ -0,0 +1,6 @@
<?php
// DO NOT EDIT
namespace Grpc\Testing;
class XdsUpdateClientConfigureServiceStub {
}

@ -25,6 +25,51 @@
$autoload_path = realpath(dirname(__FILE__).'/../../vendor/autoload.php');
require_once $autoload_path;
class XdsUpdateClientConfigureService
extends \Grpc\Testing\XdsUpdateClientConfigureServiceStub
{
function configure(\Grpc\Testing\ClientConfigureRequest $request) {
$rpc_types = $request->getTypes();
$all_metadata = $request->getMetadata();
$rpcs_to_send = [];
foreach ($rpc_types as $rpc_type) {
if ($rpc_type ==
\Grpc\Testing\ClientConfigureRequest\RpcType::EMPTY_CALL) {
$rpcs_to_send[] = 'EmptyCall';
} else if ($rpc_type ==
\Grpc\Testing\ClientConfigureRequest\RpcType::UNARY_CALL) {
$rpcs_to_send[] = 'UnaryCall';
}
}
$metadata_to_send = [];
foreach ($all_metadata as $metadata) {
$rpc_type = $metadata->getType();
if ($rpc_type ==
\Grpc\Testing\ClientConfigureRequest\RpcType::EMPTY_CALL) {
$rpc_type_key = 'EmptyCall';
} else if ($rpc_type ==
\Grpc\Testing\ClientConfigureRequest\RpcType::UNARY_CALL) {
$rpc_type_key = 'UnaryCall';
}
$key = $metadata->getKey();
$value = $metadata->getValue();
if (!isset($metadata_to_send[$rpc_type_key])) {
$metadata_to_send[$rpc_type_key] = [];
}
$metadata_to_send[$rpc_type_key][$key] = $value;
}
global $client_thread;
echo "PHP parent: Setting client_thread rpc_config to \n";
print_r($rpcs_to_send);
print_r($metadata_to_send);
echo "PHP parent: timeout_sec = ".$request->getTimeoutSec()."\n";
$client_thread->rpc_config->update($rpcs_to_send,
$metadata_to_send,
$request->getTimeoutSec());
return new Grpc\Testing\ClientConfigureResponse();
}
}
// The main xds interop test runner will ping this service to ask for
// the stats of the distribution of the backends, for the next X rpcs.
class LoadBalancerStatsService
@ -69,7 +114,8 @@ class LoadBalancerStatsService
// '', 'hostname1', 'hostname2', '', 'hostname2', ...
// ],
// ]
foreach ($client_thread->results as $rpc => $results) {
foreach ((array)$client_thread->rpc_config->rpcs_to_send as $rpc) {
$results = $client_thread->results[$rpc];
// initialize, can always start from scratch here
$rpcs_by_method[$rpc] = [];
for ($i = $start_id; $i < $end_id; $i++) {
@ -108,71 +154,204 @@ class LoadBalancerStatsService
$response->setNumFailures($num_failures);
return $response;
}
function getClientAccumulatedStats(
\Grpc\Testing\LoadBalancerAccumulatedStatsRequest $request) {
global $client_thread;
$response = new Grpc\Testing\LoadBalancerAccumulatedStatsResponse();
$response->setNumRpcsStartedByMethod(
(array)$client_thread->num_rpcs_started_by_method);
$response->setNumRpcsSucceededByMethod(
(array)$client_thread->num_rpcs_succeeded_by_method);
$response->setNumRpcsFailedByMethod(
(array)$client_thread->num_rpcs_failed_by_method);
$accumulated_method_stats
= (array)$client_thread->accumulated_method_stats;
$stats_per_method = [];
foreach ($accumulated_method_stats as $rpc_name => $stats) {
$methodStats
= new Grpc\Testing\LoadBalancerAccumulatedStatsResponse\MethodStats();
$methodStats->setRpcsStarted($stats['rpcs_started']);
$methodStats->setResult((array)$stats['result']);
$stats_per_method[$rpc_name] = $methodStats;
}
$response->setStatsPerMethod($stats_per_method);
return $response;
}
}
class RpcConfig extends Volatile {
public $server_address;
public $qps;
public $fail_on_failed_rpcs;
public $rpcs_to_send;
public $metadata_to_send;
public $tmp_file1;
public $tmp_file2;
public $timeout_sec;
public function __construct($server_address,
$qps,
$fail_on_failed_rpcs,
$rpcs_to_send,
$metadata_to_send,
$tmp_file1,
$tmp_file2) {
$this->server_address = $server_address;
$this->qps = $qps;
$this->fail_on_failed_rpcs = $fail_on_failed_rpcs;
$this->rpcs_to_send = (array)$rpcs_to_send;
$this->metadata_to_send = (array)$metadata_to_send;
$this->tmp_file1 = $tmp_file1;
$this->tmp_file2 = $tmp_file2;
$this->timeout_sec = 30;
}
public function update($rpcs_to_send, $metadata_to_send, $timeout_sec) {
$this->rpcs_to_send = (array)$rpcs_to_send;
$this->metadata_to_send = (array)$metadata_to_send;
$this->timeout_sec = $timeout_sec;
}
}
// This client thread blindly sends a unary RPC to the server once
// every 1 / qps seconds.
class ClientThread extends Thread {
private $server_address_;
private $target_seconds_between_rpcs_;
private $fail_on_failed_rpcs_;
private $autoload_path_;
private $TIMEOUT_US = 30 * 1e6; // 30 seconds
public $rpc_config;
public $num_results = 0;
public $results;
public function __construct($server_address, $qps, $fail_on_failed_rpcs,
$rpcs_to_send, $metadata_to_send,
public $RPC_MAP = [
'UnaryCall' => 'UNARY_CALL',
'EmptyCall' => 'EMPTY_CALL',
];
public $num_rpcs_started_by_method = [];
public $num_rpcs_succeeded_by_method = [];
public $num_rpcs_failed_by_method = [];
public $accumulated_method_stats = [];
public function __construct($rpc_config,
$autoload_path) {
$this->server_address_ = $server_address;
$this->target_seconds_between_rpcs_ = 1.0 / $qps;
$this->fail_on_failed_rpcs_ = $fail_on_failed_rpcs;
$this->rpcs_to_send = explode(',', $rpcs_to_send);
// Convert input in the form of
// rpc1:k1:v1,rpc2:k2:v2,rpc1:k3:v3
// into
// [
// 'rpc1' => [
// 'k1' => 'v1',
// 'k3' => 'v3',
// ],
// 'rpc2' => [
// 'k2' => 'v2'
// ],
// ]
$this->metadata_to_send = [];
if ($_all_metadata = explode(',', $metadata_to_send)) {
foreach ($_all_metadata as $one_metadata_pair) {
list($rpc,
$metadata_key,
$metadata_value) = explode(':', $one_metadata_pair);
// initialize in case we haven't seen this rpc before
if (!array_key_exists($rpc, $this->metadata_to_send)) {
$this->metadata_to_send[$rpc] = [];
}
$this->metadata_to_send[$rpc][$metadata_key]
= $metadata_value;
}
}
$this->rpc_config = $rpc_config;
$this->target_seconds_between_rpcs_ = 1.0 / $rpc_config->qps;
$this->autoload_path_ = $autoload_path;
$this->simple_request = new Grpc\Testing\SimpleRequest();
$this->empty_request = new Grpc\Testing\EmptyMessage();
$this->results = [];
foreach ($this->rpcs_to_send as $rpc) {
foreach (['UnaryCall', 'EmptyCall'] as $rpc) {
$this->results[$rpc] = [];
}
$this->outstanding_rpcs = [];
foreach (['UNARY_CALL', 'EMPTY_CALL'] as $rpc_stats_key) {
$this->num_rpcs_started_by_method[$rpc_stats_key] = 0;
$this->num_rpcs_succeeded_by_method[$rpc_stats_key] = 0;
$this->num_rpcs_failed_by_method[$rpc_stats_key] = 0;
$this->accumulated_method_stats[$rpc_stats_key] = [
'rpcs_started' => 0,
'result' => [],
];
}
}
public function sendUnaryCall($stub, $metadata) {
$timeout = $this->rpc_config->timeout_sec ?
$this->rpc_config->timeout_sec * 1e6 :
$this->TIMEOUT_US;
return $stub->UnaryCall($this->simple_request,
$metadata,
['timeout' => $this->TIMEOUT_US]);
['timeout' => $timeout]);
}
public function sendEmptyCall($stub, $metadata) {
$timeout = $this->rpc_config->timeout_sec ?
$this->rpc_config->timeout_sec * 1e6 :
$this->TIMEOUT_US;
return $stub->EmptyCall($this->empty_request,
$metadata,
['timeout' => $this->TIMEOUT_US]);
['timeout' => $timeout]);
}
public function add_rpc_result($rpc, $status_code) {
// $rpc here needs to be in the format of 'UnaryCall', 'EmptyCall'
if (!isset($this->accumulated_method_stats[$this->RPC_MAP[$rpc]]
['result'][$status_code])) {
$this->accumulated_method_stats[$this->RPC_MAP[$rpc]]
['result'][$status_code] = 0;
}
$this->accumulated_method_stats[$this->RPC_MAP[$rpc]]
['result'][$status_code] += 1;
}
public function check_child_process_result() {
if (sizeof($this->outstanding_rpcs) > 0 &&
$this->rpc_config->tmp_file2) {
$keys_to_delete = [];
// tmp_file2 contains the RPC result of each RPC we
// originally wrote to tmp_file1
$f2 = fopen($this->rpc_config->tmp_file2, 'r+');
flock($f2, LOCK_EX);
while (true) {
$f2_line = fgets($f2);
if (!$f2_line) {
break;
}
// format here needs to be in sync with
// src/php/bin/xds_manager.py
$parts = explode(',', trim($f2_line));
$key = $parts[0];
$returncode = $parts[1];
if (isset($this->outstanding_rpcs[$key])) {
$parts2 = explode('|', $key);
$result_num = $parts2[0];
$rpc_name = $parts2[1];
// Child processes can only communicate back the
// status code for now.
// Current interop test specs only call for
// reporting back the status code in these scenarios.
// If we ever need the hostname reported back from
// child processes, we need to enhance this
// communication framework through tmp files.
$this->results[$rpc_name][$result_num] = "";
if ($returncode) {
$this->num_rpcs_failed_by_method
[$this->RPC_MAP[$rpc_name]] += 1;
} else {
$this->num_rpcs_succeeded_by_method
[$this->RPC_MAP[$rpc_name]] += 1;
}
$this->add_rpc_result($rpc_name, $returncode);
$keys_to_delete[] = $key;
}
}
foreach ($keys_to_delete as $key) {
unset($this->outstanding_rpcs[$key]);
}
ftruncate($f2, 0);
flock($f2, LOCK_UN);
fclose($f2);
}
}
public function execute_rpc_in_child_process($rpc, $rpc_behavior) {
// tmp_file1 contains the list of RPCs (and their
// specs) we want executed. This will be picked up
// by src/php/bin/xds_manager.py
$f1 = fopen($this->rpc_config->tmp_file1, 'a');
$key = implode('|', [$this->num_results,
$rpc,
$rpc_behavior,
$this->rpc_config->timeout_sec]);
flock($f1, LOCK_EX);
fwrite($f1, $key."\n");
fflush($f1);
flock($f1, LOCK_UN);
fclose($f1);
$this->outstanding_rpcs[$key] = 1;
$this->num_rpcs_started_by_method[$this->RPC_MAP[$rpc]] += 1;
$this->accumulated_method_stats[$this->RPC_MAP[$rpc]]
['rpcs_started'] += 1;
}
public function run() {
@ -180,10 +359,11 @@ class ClientThread extends Thread {
// Hence we need to do this.
require_once($this->autoload_path_);
$stub = new Grpc\Testing\TestServiceClient($this->server_address_, [
'credentials' => Grpc\ChannelCredentials::createInsecure()
$stub = new Grpc\Testing\TestServiceClient(
$this->rpc_config->server_address,
['credentials' => Grpc\ChannelCredentials::createInsecure()
]);
# hrtime returns nanoseconds
// hrtime returns nanoseconds
$target_next_start_us = hrtime(true) / 1000;
while (true) {
$now_us = hrtime(true) / 1000;
@ -191,27 +371,35 @@ class ClientThread extends Thread {
if ($sleep_us < 0) {
$target_next_start_us =
$now_us + ($this->target_seconds_between_rpcs_ * 1e6);
echo sprintf(
"php xds: warning, rpc takes too long to finish. "
. "Deficit %.1fms."
. "If you consistently see this, the qps is too high.\n",
round(abs($sleep_us / 1000), 1));
} else {
$target_next_start_us +=
($this->target_seconds_between_rpcs_ * 1e6);
usleep($sleep_us);
}
foreach ($this->rpcs_to_send as $rpc) {
$metadata = array_key_exists(
$rpc, $this->metadata_to_send) ?
$this->metadata_to_send[$rpc] : [];
$this->check_child_process_result();
foreach ($this->rpc_config->rpcs_to_send as $rpc) {
$metadata_to_send_arr
= (array)$this->rpc_config->metadata_to_send;
$metadata = array_key_exists($rpc, $metadata_to_send_arr) ?
$metadata_to_send_arr[$rpc] : [];
// This copy is somehow necessary because
// $this->metadata_to_send[$rpc] somehow becomes a
// Volatile object, instead of an associative array.
$metadata_array = [];
$rpc_behavior = false;
foreach ($metadata as $key => $value) {
$metadata_array[$key] = [$value];
if ($key == 'rpc-behavior') {
$rpc_behavior = $value;
}
}
if ($rpc_behavior && $this->rpc_config->tmp_file1) {
// if 'rpc-behavior' is set, we need to pawn off
// the execution to some other child PHP processes
$this->execute_rpc_in_child_process($rpc, $rpc_behavior);
continue;
}
// Execute RPC within this script
$call = null;
if ($rpc == 'UnaryCall') {
$call = $this->sendUnaryCall($stub, $metadata_array);
@ -220,19 +408,29 @@ class ClientThread extends Thread {
} else {
throw new Exception("Unhandled rpc $rpc");
}
$this->num_rpcs_started_by_method[$this->RPC_MAP[$rpc]] += 1;
$this->accumulated_method_stats[$this->RPC_MAP[$rpc]]
['rpcs_started'] += 1;
// the remote peer is being returned as part of the
// initial metadata, according to the test spec
$initial_metadata = $call->getMetadata();
list($response, $status) = $call->wait();
if ($status->code == Grpc\STATUS_OK &&
array_key_exists('hostname', $initial_metadata)) {
$this->results[$rpc][] = $initial_metadata['hostname'][0];
$this->results[$rpc][$this->num_results]
= $initial_metadata['hostname'][0];
$this->num_rpcs_succeeded_by_method
[$this->RPC_MAP[$rpc]] += 1;
$this->add_rpc_result($rpc, 0);
} else {
if ($this->fail_on_failed_rpcs_) {
if ($this->rpc_config->fail_on_failed_rpcs_) {
throw new Exception("$rpc failed with status "
. $status->code);
}
$this->results[$rpc][] = "";
$this->results[$rpc][$this->num_results] = "";
$this->num_rpcs_failed_by_method
[$this->RPC_MAP[$rpc]] += 1;
$this->add_rpc_result($rpc, $status->code);
}
}
// $num_results here is only incremented when the group of
@ -250,18 +448,58 @@ class ClientThread extends Thread {
// Note: num_channels are currently ignored for now
$args = getopt('', ['fail_on_failed_rpcs:', 'num_channels:',
'rpc:', 'metadata:',
'rpc:', 'metadata:', 'tmp_file1:', 'tmp_file2:',
'server:', 'stats_port:', 'qps:']);
$client_thread = new ClientThread($args['server'], $args['qps'],
$args['fail_on_failed_rpcs'],
(empty($args['rpc']) ? 'UnaryCall'
: $args['rpc']),
$args['metadata'],
// Convert input in the form of
// rpc1:k1:v1,rpc2:k2:v2,rpc1:k3:v3
// into
// [
// 'rpc1' => [
// 'k1' => 'v1',
// 'k3' => 'v3',
// ],
// 'rpc2' => [
// 'k2' => 'v2'
// ],
// ]
$metadata_to_send = [];
if ($_all_metadata = explode(',', $args['metadata'])) {
foreach ($_all_metadata as $one_metadata_pair) {
list($rpc,
$metadata_key,
$metadata_value) = explode(':', $one_metadata_pair);
// initialize in case we haven't seen this rpc before
if (!array_key_exists($rpc, $metadata_to_send)) {
$metadata_to_send[$rpc] = [];
}
$metadata_to_send[$rpc][$metadata_key] = $metadata_value;
}
}
$rpcs_to_send = (empty($args['rpc']) ? 'UnaryCall' : $args['rpc']);
// Need to communicate the xds server name to the async runner manager
if ($args['tmp_file1']) {
$f1 = fopen($args['tmp_file1'], 'w');
fwrite($f1, 'server_address,'.$args['server']);
fclose($f1);
}
$rpc_config = new RpcConfig($args['server'],
$args['qps'],
$args['fail_on_failed_rpcs'],
explode(',', $rpcs_to_send),
$metadata_to_send,
$args['tmp_file1'],
$args['tmp_file2']);
$client_thread = new ClientThread($rpc_config,
$autoload_path);
$client_thread->start();
$server = new Grpc\RpcServer();
$server->addHttp2Port('0.0.0.0:'.$args['stats_port']);
$server->handle(new LoadBalancerStatsService());
$server->handle(new XdsUpdateClientConfigureService());
$server->run();

@ -0,0 +1,54 @@
<?php
/*
*
* Copyright 2021 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.
*
*/
$autoload_path = realpath(dirname(__FILE__).'/../../vendor/autoload.php');
require_once $autoload_path;
// This script is used to launch 1 single EmptyCall RPC, most likely
// for the purpose of starting such RPC asynchronously away from the
// main PHP xDS interop client src/php/tests/interop/xds_client.php.
// This script is launched from src/php/bin/xds_manager.py. The result
// of this RPC will be aggregated and reported back to the main runner
// from there.
$args = getopt('', ['server:', 'num:',
'rpc_behavior:', 'timeout_sec:']);
$TIMEOUT_US = 30 * 1e6; // 30 seconds
$server_address = $args['server'];
$num = $args['num'];
$stub = new Grpc\Testing\TestServiceClient($server_address, [
'credentials' => Grpc\ChannelCredentials::createInsecure()
]);
$empty_request = new Grpc\Testing\EmptyMessage();
$timeout = $args['timeout_sec'] ? $args['timeout_sec'] * 1e6 : $TIMEOUT_US;
$metadata = [];
if ($args['rpc_behavior']) {
$metadata['rpc_behavior'] = [$args['rpc_behavior']];
}
$call = $stub->EmptyCall($empty_request,
$metadata,
['timeout' => $timeout]);
list($response, $status) = $call->wait();
exit($status->code);

@ -0,0 +1,54 @@
<?php
/*
*
* Copyright 2021 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.
*
*/
$autoload_path = realpath(dirname(__FILE__).'/../../vendor/autoload.php');
require_once $autoload_path;
// This script is used to launch 1 single EmptyCall RPC, most likely
// for the purpose of starting such RPC asynchronously away from the
// main PHP xDS interop client src/php/tests/interop/xds_client.php.
// This script is launched from src/php/bin/xds_manager.py. The result
// of this RPC will be aggregated and reported back to the main runner
// from there.
$args = getopt('', ['server:', 'num:',
'rpc_behavior:', 'timeout_sec:']);
$TIMEOUT_US = 30 * 1e6; // 30 seconds
$server_address = $args['server'];
$num = $args['num'];
$stub = new Grpc\Testing\TestServiceClient($server_address, [
'credentials' => Grpc\ChannelCredentials::createInsecure()
]);
$simple_request = new Grpc\Testing\SimpleRequest();
$timeout = $args['timeout_sec'] ? $args['timeout_sec'] * 1e6 : $TIMEOUT_US;
$metadata = [];
if ($args['rpc_behavior']) {
$metadata['rpc-behavior'] = [$args['rpc_behavior']];
}
$call = $stub->UnaryCall($simple_request,
$metadata,
['timeout' => $timeout]);
list($response, $status) = $call->wait();
exit($status->code);

@ -70,14 +70,27 @@ export CC=/usr/bin/gcc
composer install && \
./bin/generate_proto_php.sh)
GRPC_VERBOSITY=debug GRPC_TRACE=xds_client,xds_resolver,xds_cluster_manager_lb,cds_lb,xds_cluster_resolver_lb,priority_lb,xds_cluster_impl_lb,weighted_target_lb "$PYTHON" \
tools/run_tests/run_xds_tests.py \
--test_case="timeout" \
--project_id=grpc-testing \
--project_num=830293263384 \
--source_image=projects/grpc-testing/global/images/xds-test-server-4 \
--path_to_server_binary=/java_server/grpc-java/interop-testing/build/install/grpc-interop-testing/bin/xds-test-server \
--gcp_suffix=$(date '+%s') \
--verbose \
--qps=20 \
${XDS_V3_OPT-} \
--client_cmd='./src/php/bin/run_xds_client.sh --server=xds:///{server_uri} --stats_port={stats_port} --qps={qps} {fail_on_failed_rpc} {rpcs_to_send} {metadata_to_send}'
GRPC_VERBOSITY=debug GRPC_TRACE=xds_client,xds_resolver,xds_cluster_manager_lb,cds_lb,xds_cluster_resolver_lb,priority_lb,xds_cluster_impl_lb,weighted_target_lb "$PYTHON" \
tools/run_tests/run_xds_tests.py \
--test_case="all,path_matching,header_matching" \
--project_id=grpc-testing \
--project_num=830293263384 \
--source_image=projects/grpc-testing/global/images/xds-test-server-2 \
--source_image=projects/grpc-testing/global/images/xds-test-server-4 \
--path_to_server_binary=/java_server/grpc-java/interop-testing/build/install/grpc-interop-testing/bin/xds-test-server \
--gcp_suffix=$(date '+%s') \
--verbose \
${XDS_V3_OPT-} \
--client_cmd='php -d extension=grpc.so -d extension=pthreads.so src/php/tests/interop/xds_client.php --server=xds:///{server_uri} --stats_port={stats_port} --qps={qps} {fail_on_failed_rpc} {rpcs_to_send} {metadata_to_send}'
--client_cmd='./src/php/bin/run_xds_client.sh --server=xds:///{server_uri} --stats_port={stats_port} --qps={qps} {fail_on_failed_rpc} {rpcs_to_send} {metadata_to_send}'

Loading…
Cancel
Save