Merge github.com:grpc/grpc into clangf

pull/3556/head
Craig Tiller 9 years ago
commit b7a597771f
  1. 4
      src/compiler/objective_c_generator.cc
  2. 5
      src/core/iomgr/exec_ctx.c
  3. 5
      src/core/iomgr/exec_ctx.h
  4. 14
      src/core/iomgr/iocp_windows.c
  5. 1
      src/core/iomgr/iocp_windows.h
  6. 2
      src/core/iomgr/iomgr.c
  7. 3
      src/core/iomgr/iomgr_internal.h
  8. 2
      src/core/iomgr/iomgr_posix.c
  9. 2
      src/core/iomgr/iomgr_windows.c
  10. 10
      src/core/iomgr/pollset_multipoller_with_epoll.c
  11. 2
      src/core/iomgr/pollset_multipoller_with_poll_posix.c
  12. 9
      src/core/iomgr/pollset_posix.c
  13. 58
      src/core/iomgr/pollset_windows.c
  14. 14
      src/core/iomgr/pollset_windows.h
  15. 4
      src/core/iomgr/tcp_server_posix.c
  16. 6
      src/core/iomgr/udp_server.c
  17. 2
      src/core/iomgr/udp_server.h
  18. 2
      src/core/support/histogram.c
  19. 14
      src/core/support/time_posix.c
  20. 12
      src/core/transport/chttp2/hpack_table.c
  21. 12
      src/core/transport/chttp2/stream_encoder.c
  22. 6
      src/core/transport/stream_op.c
  23. 15
      src/cpp/client/secure_credentials.cc
  24. 24
      src/cpp/server/secure_server_credentials.cc
  25. 4
      src/csharp/Grpc.Core.Tests/Internal/CompletionQueueSafeHandleTest.cs
  26. 2
      src/csharp/Grpc.IntegrationTesting/InteropClient.cs
  27. 6
      src/node/.istanbul.yml
  28. 15
      src/node/binding.gyp
  29. 6
      src/node/package.json
  30. 12
      src/objective-c/tests/InteropTests.m
  31. 60
      src/python/grpcio_test/grpc_test/conftest.py
  32. 2
      test/core/end2end/README
  33. 4
      test/core/iomgr/udp_server_test.c
  34. 34
      test/core/util/test_config.c
  35. 896
      test/cpp/end2end/end2end_test.cc
  36. 32
      test/cpp/end2end/streaming_throughput_test.cc
  37. 111
      tools/jenkins/build_docker_and_run_interop_tests.sh
  38. 79
      tools/jenkins/docker_prepare_interop_tests.sh
  39. 42
      tools/jenkins/docker_run_interop_servers.sh
  40. 39
      tools/jenkins/docker_run_interop_tests.sh
  41. 3
      tools/jenkins/docker_run_tests.sh
  42. 3
      tools/jenkins/grpc_jenkins_slave/Dockerfile
  43. 3
      tools/jenkins/grpc_jenkins_slave_32bits/Dockerfile
  44. 2
      tools/jenkins/run_distribution.sh
  45. 9
      tools/jenkins/run_jenkins.sh
  46. 330
      tools/run_tests/run_interop_tests.py
  47. 37
      tools/run_tests/run_interops.py
  48. 75
      tools/run_tests/run_interops_build.sh
  49. 15
      tools/run_tests/run_node.sh
  50. 10
      tools/run_tests/run_python.sh
  51. 2
      tools/run_tests/run_tests.py
  52. 35
      vsprojects/coapp/zlib/README.md
  53. 51
      vsprojects/coapp/zlib/buildall.bat
  54. 102
      vsprojects/coapp/zlib/grpc.dependencies.zlib.autopkg
  55. 13
      vsprojects/coapp/zlib/managed_targets/grpc.dependencies.zlib.redist.props
  56. 14
      vsprojects/coapp/zlib/managed_targets/grpc.dependencies.zlib.redist.targets
  57. 1
      vsprojects/coapp/zlib/version.inc
  58. 26
      vsprojects/coapp/zlib/zlib.sln
  59. 147
      vsprojects/coapp/zlib/zlib.vcxproj

@ -203,6 +203,7 @@ void PrintMethodImplementations(Printer *printer,
printer.Print(
"- (instancetype)initWithHost:(NSString *)host"
" NS_DESIGNATED_INITIALIZER;\n");
printer.Print("+ (instancetype)serviceWithHost:(NSString *)host;\n");
printer.Print("@end\n");
}
return output;
@ -239,6 +240,9 @@ void PrintMethodImplementations(Printer *printer,
printer.Print(" packageName:(NSString *)packageName\n");
printer.Print(" serviceName:(NSString *)serviceName {\n");
printer.Print(" return [self initWithHost:host];\n");
printer.Print("}\n\n");
printer.Print("+ (instancetype)serviceWithHost:(NSString *)host {\n");
printer.Print(" return [[self alloc] initWithHost:host];\n");
printer.Print("}\n\n\n");
for (int i = 0; i < service->method_count(); i++) {

@ -35,16 +35,19 @@
#include <grpc/support/log.h>
void grpc_exec_ctx_flush(grpc_exec_ctx *exec_ctx) {
int grpc_exec_ctx_flush(grpc_exec_ctx *exec_ctx) {
int did_something = 0;
while (!grpc_closure_list_empty(exec_ctx->closure_list)) {
grpc_closure *c = exec_ctx->closure_list.head;
exec_ctx->closure_list.head = exec_ctx->closure_list.tail = NULL;
while (c != NULL) {
grpc_closure *next = c->next;
did_something = 1;
c->cb(exec_ctx, c->cb_arg, c->success);
c = next;
}
}
return did_something;
}
void grpc_exec_ctx_finish(grpc_exec_ctx *exec_ctx) {

@ -61,8 +61,9 @@ struct grpc_exec_ctx {
{ GRPC_CLOSURE_LIST_INIT }
/** Flush any work that has been enqueued onto this grpc_exec_ctx.
* Caller must guarantee that no interfering locks are held. */
void grpc_exec_ctx_flush(grpc_exec_ctx *exec_ctx);
* Caller must guarantee that no interfering locks are held.
* Returns 1 if work was performed, 0 otherwise. */
int grpc_exec_ctx_flush(grpc_exec_ctx *exec_ctx);
/** Finish any pending work for a grpc_exec_ctx. Must be called before
* the instance is destroyed, or work may be lost. */
void grpc_exec_ctx_finish(grpc_exec_ctx *exec_ctx);

@ -68,7 +68,7 @@ static DWORD deadline_to_millis_timeout(gpr_timespec deadline,
}
timeout = gpr_time_sub(deadline, now);
return gpr_time_to_millis(gpr_time_add(
timeout, gpr_time_from_nanos(GPR_NS_PER_SEC - 1, GPR_TIMESPAN)));
timeout, gpr_time_from_nanos(GPR_NS_PER_MS - 1, GPR_TIMESPAN)));
}
void grpc_iocp_work(grpc_exec_ctx *exec_ctx, gpr_timespec deadline) {
@ -120,9 +120,7 @@ void grpc_iocp_work(grpc_exec_ctx *exec_ctx, gpr_timespec deadline) {
info->has_pending_iocp = 1;
}
gpr_mu_unlock(&socket->state_mu);
if (closure) {
closure->cb(exec_ctx, closure->cb_arg, 1);
}
grpc_exec_ctx_enqueue(exec_ctx, closure, 1);
}
void grpc_iocp_init(void) {
@ -140,6 +138,14 @@ void grpc_iocp_kick(void) {
GPR_ASSERT(success);
}
void grpc_iocp_flush(void) {
grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
do {
grpc_iocp_work(&exec_ctx, gpr_inf_past(GPR_CLOCK_MONOTONIC));
} while (grpc_exec_ctx_flush(&exec_ctx));
}
void grpc_iocp_shutdown(void) {
grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
while (gpr_atm_acq_load(&g_custom_events)) {

@ -41,6 +41,7 @@
void grpc_iocp_work(grpc_exec_ctx *exec_ctx, gpr_timespec deadline);
void grpc_iocp_init(void);
void grpc_iocp_kick(void);
void grpc_iocp_flush(void);
void grpc_iocp_shutdown(void);
void grpc_iocp_add_socket(grpc_winsocket *);

@ -84,6 +84,8 @@ void grpc_iomgr_shutdown(void) {
gpr_timespec last_warning_time = gpr_now(GPR_CLOCK_REALTIME);
grpc_exec_ctx exec_ctx = GRPC_EXEC_CTX_INIT;
grpc_iomgr_platform_flush();
gpr_mu_lock(&g_mu);
g_shutdown = 1;
while (g_root_object.next != &g_root_object) {

@ -50,6 +50,9 @@ void grpc_iomgr_register_object(grpc_iomgr_object *obj, const char *name);
void grpc_iomgr_unregister_object(grpc_iomgr_object *obj);
void grpc_iomgr_platform_init(void);
/** flush any globally queued work from iomgr */
void grpc_iomgr_platform_flush(void);
/** tear down all platform specific global iomgr structures */
void grpc_iomgr_platform_shutdown(void);
#endif /* GRPC_INTERNAL_CORE_IOMGR_IOMGR_INTERNAL_H */

@ -45,6 +45,8 @@ void grpc_iomgr_platform_init(void) {
grpc_register_tracer("tcp", &grpc_tcp_trace);
}
void grpc_iomgr_platform_flush(void) {}
void grpc_iomgr_platform_shutdown(void) { grpc_fd_global_shutdown(); }
#endif /* GRPC_POSIX_SOCKET */

@ -63,6 +63,8 @@ void grpc_iomgr_platform_init(void) {
grpc_iocp_init();
}
void grpc_iomgr_platform_flush(void) { grpc_iocp_flush(); }
void grpc_iomgr_platform_shutdown(void) {
grpc_iocp_shutdown();
winsock_shutdown();

@ -180,6 +180,8 @@ static void multipoll_with_epoll_pollset_maybe_work_and_unlock(
pfds[1].events = POLLIN;
pfds[1].revents = 0;
/* TODO(vpai): Consider first doing a 0 timeout poll here to avoid
even going into the blocking annotation if possible */
GRPC_SCHEDULING_START_BLOCKING_REGION;
poll_rv = grpc_poll_function(pfds, 2, timeout_ms);
GRPC_SCHEDULING_END_BLOCKING_REGION;
@ -209,15 +211,15 @@ static void multipoll_with_epoll_pollset_maybe_work_and_unlock(
/* TODO(klempner): We might want to consider making err and pri
* separate events */
int cancel = ep_ev[i].events & (EPOLLERR | EPOLLHUP);
int read = ep_ev[i].events & (EPOLLIN | EPOLLPRI);
int write = ep_ev[i].events & EPOLLOUT;
int read_ev = ep_ev[i].events & (EPOLLIN | EPOLLPRI);
int write_ev = ep_ev[i].events & EPOLLOUT;
if (fd == NULL) {
grpc_wakeup_fd_consume_wakeup(&grpc_global_wakeup_fd);
} else {
if (read || cancel) {
if (read_ev || cancel) {
grpc_fd_become_readable(exec_ctx, fd);
}
if (write || cancel) {
if (write_ev || cancel) {
grpc_fd_become_writable(exec_ctx, fd);
}
}

@ -151,6 +151,8 @@ static void multipoll_with_poll_pollset_maybe_work_and_unlock(
POLLOUT, &watchers[i]);
}
/* TODO(vpai): Consider first doing a 0 timeout poll here to avoid
even going into the blocking annotation if possible */
GRPC_SCHEDULING_START_BLOCKING_REGION;
r = grpc_poll_function(pfds, pfd_count, timeout);
GRPC_SCHEDULING_END_BLOCKING_REGION;

@ -57,7 +57,14 @@
GPR_TLS_DECL(g_current_thread_poller);
GPR_TLS_DECL(g_current_thread_worker);
/** Default poll() function - a pointer so that it can be overridden by some
* tests */
grpc_poll_function_type grpc_poll_function = poll;
/** The alarm system needs to be able to wakeup 'some poller' sometimes
* (specifically when a new alarm needs to be triggered earlier than the next
* alarm 'epoch').
* This wakeup_fd gives us something to alert on when such a case occurs. */
grpc_wakeup_fd grpc_global_wakeup_fd;
static void remove_worker(grpc_pollset *p, grpc_pollset_worker *worker) {
@ -479,6 +486,8 @@ static void basic_pollset_maybe_work_and_unlock(grpc_exec_ctx *exec_ctx,
gpr_mu_unlock(&pollset->mu);
}
/* TODO(vpai): Consider first doing a 0 timeout poll here to avoid
even going into the blocking annotation if possible */
/* poll fd count (argument 2) is shortened by one if we have no events
to poll on - such that it only includes the kicker */
GRPC_SCHEDULING_START_BLOCKING_REGION;

@ -43,19 +43,19 @@
#include "src/core/iomgr/pollset.h"
#include "src/core/iomgr/pollset_windows.h"
static gpr_mu g_polling_mu;
gpr_mu grpc_polling_mu;
static grpc_pollset_worker *g_active_poller;
static grpc_pollset_worker g_global_root_worker;
void grpc_pollset_global_init() {
gpr_mu_init(&g_polling_mu);
gpr_mu_init(&grpc_polling_mu);
g_active_poller = NULL;
g_global_root_worker.links[GRPC_POLLSET_WORKER_LINK_GLOBAL].next =
g_global_root_worker.links[GRPC_POLLSET_WORKER_LINK_GLOBAL].prev =
&g_global_root_worker;
}
void grpc_pollset_global_shutdown() { gpr_mu_destroy(&g_polling_mu); }
void grpc_pollset_global_shutdown() { gpr_mu_destroy(&grpc_polling_mu); }
static void remove_worker(grpc_pollset_worker *worker,
grpc_pollset_worker_link_type type) {
@ -105,7 +105,6 @@ static void push_front_worker(grpc_pollset_worker *root,
void grpc_pollset_init(grpc_pollset *pollset) {
memset(pollset, 0, sizeof(*pollset));
gpr_mu_init(&pollset->mu);
pollset->root_worker.links[GRPC_POLLSET_WORKER_LINK_POLLSET].next =
pollset->root_worker.links[GRPC_POLLSET_WORKER_LINK_POLLSET].prev =
&pollset->root_worker;
@ -113,7 +112,7 @@ void grpc_pollset_init(grpc_pollset *pollset) {
void grpc_pollset_shutdown(grpc_exec_ctx *exec_ctx, grpc_pollset *pollset,
grpc_closure *closure) {
gpr_mu_lock(&pollset->mu);
gpr_mu_lock(&grpc_polling_mu);
pollset->shutting_down = 1;
grpc_pollset_kick(pollset, GRPC_POLLSET_KICK_BROADCAST);
if (!pollset->is_iocp_worker) {
@ -121,43 +120,49 @@ void grpc_pollset_shutdown(grpc_exec_ctx *exec_ctx, grpc_pollset *pollset,
} else {
pollset->on_shutdown = closure;
}
gpr_mu_unlock(&pollset->mu);
gpr_mu_unlock(&grpc_polling_mu);
}
void grpc_pollset_destroy(grpc_pollset *pollset) {
gpr_mu_destroy(&pollset->mu);
}
void grpc_pollset_destroy(grpc_pollset *pollset) {}
void grpc_pollset_work(grpc_exec_ctx *exec_ctx, grpc_pollset *pollset,
grpc_pollset_worker *worker, gpr_timespec now,
gpr_timespec deadline) {
int added_worker = 0;
worker->links[GRPC_POLLSET_WORKER_LINK_POLLSET].next =
worker->links[GRPC_POLLSET_WORKER_LINK_POLLSET].prev = NULL;
worker->links[GRPC_POLLSET_WORKER_LINK_POLLSET].prev =
worker->links[GRPC_POLLSET_WORKER_LINK_GLOBAL].next =
worker->links[GRPC_POLLSET_WORKER_LINK_GLOBAL].prev = NULL;
worker->kicked = 0;
worker->pollset = pollset;
gpr_cv_init(&worker->cv);
if (grpc_alarm_check(exec_ctx, now, &deadline)) {
goto done;
}
if (!pollset->kicked_without_pollers && !pollset->shutting_down) {
gpr_mu_lock(&g_polling_mu);
if (g_active_poller == NULL) {
grpc_pollset_worker *next_worker;
/* become poller */
pollset->is_iocp_worker = 1;
g_active_poller = worker;
gpr_mu_unlock(&g_polling_mu);
gpr_mu_unlock(&pollset->mu);
gpr_mu_unlock(&grpc_polling_mu);
grpc_iocp_work(exec_ctx, deadline);
gpr_mu_lock(&pollset->mu);
gpr_mu_lock(&g_polling_mu);
grpc_exec_ctx_flush(exec_ctx);
gpr_mu_lock(&grpc_polling_mu);
pollset->is_iocp_worker = 0;
g_active_poller = NULL;
next_worker = pop_front_worker(&g_global_root_worker,
GRPC_POLLSET_WORKER_LINK_GLOBAL);
/* try to get a worker from this pollsets worker list */
next_worker = pop_front_worker(&pollset->root_worker,
GRPC_POLLSET_WORKER_LINK_POLLSET);
if (next_worker == NULL) {
/* try to get a worker from the global list */
next_worker = pop_front_worker(&g_global_root_worker,
GRPC_POLLSET_WORKER_LINK_GLOBAL);
}
if (next_worker != NULL) {
next_worker->kicked = 1;
gpr_cv_signal(&next_worker->cv);
}
gpr_mu_unlock(&g_polling_mu);
if (pollset->shutting_down && pollset->on_shutdown != NULL) {
grpc_exec_ctx_enqueue(exec_ctx, pollset->on_shutdown, 1);
@ -167,25 +172,28 @@ void grpc_pollset_work(grpc_exec_ctx *exec_ctx, grpc_pollset *pollset,
}
push_front_worker(&g_global_root_worker, GRPC_POLLSET_WORKER_LINK_GLOBAL,
worker);
gpr_mu_unlock(&g_polling_mu);
push_front_worker(&pollset->root_worker, GRPC_POLLSET_WORKER_LINK_POLLSET,
worker);
added_worker = 1;
gpr_cv_wait(&worker->cv, &pollset->mu, deadline);
while (!worker->kicked) {
if (gpr_cv_wait(&worker->cv, &grpc_polling_mu, deadline)) {
break;
}
}
} else {
pollset->kicked_without_pollers = 0;
}
done:
if (!grpc_closure_list_empty(exec_ctx->closure_list)) {
gpr_mu_unlock(&pollset->mu);
gpr_mu_unlock(&grpc_polling_mu);
grpc_exec_ctx_flush(exec_ctx);
gpr_mu_lock(&pollset->mu);
gpr_mu_lock(&grpc_polling_mu);
}
gpr_cv_destroy(&worker->cv);
if (added_worker) {
remove_worker(worker, GRPC_POLLSET_WORKER_LINK_GLOBAL);
remove_worker(worker, GRPC_POLLSET_WORKER_LINK_POLLSET);
}
gpr_cv_destroy(&worker->cv);
}
void grpc_pollset_kick(grpc_pollset *p, grpc_pollset_worker *specific_worker) {
@ -196,6 +204,7 @@ void grpc_pollset_kick(grpc_pollset *p, grpc_pollset_worker *specific_worker) {
specific_worker != &p->root_worker;
specific_worker =
specific_worker->links[GRPC_POLLSET_WORKER_LINK_POLLSET].next) {
specific_worker->kicked = 1;
gpr_cv_signal(&specific_worker->cv);
}
p->kicked_without_pollers = 1;
@ -204,12 +213,11 @@ void grpc_pollset_kick(grpc_pollset *p, grpc_pollset_worker *specific_worker) {
}
} else {
if (p->is_iocp_worker) {
gpr_mu_lock(&g_polling_mu);
if (g_active_poller == specific_worker) {
grpc_iocp_kick();
}
gpr_mu_unlock(&g_polling_mu);
} else {
specific_worker->kicked = 1;
gpr_cv_signal(&specific_worker->cv);
}
}

@ -54,20 +54,26 @@ typedef struct grpc_pollset_worker_link {
struct grpc_pollset_worker *prev;
} grpc_pollset_worker_link;
struct grpc_pollset;
typedef struct grpc_pollset grpc_pollset;
typedef struct grpc_pollset_worker {
gpr_cv cv;
int kicked;
struct grpc_pollset *pollset;
grpc_pollset_worker_link links[GRPC_POLLSET_WORKER_LINK_TYPES];
} grpc_pollset_worker;
typedef struct grpc_pollset {
gpr_mu mu;
struct grpc_pollset {
int shutting_down;
int kicked_without_pollers;
int is_iocp_worker;
grpc_pollset_worker root_worker;
grpc_closure *on_shutdown;
} grpc_pollset;
};
extern gpr_mu grpc_polling_mu;
#define GRPC_POLLSET_MU(pollset) (&(pollset)->mu)
#define GRPC_POLLSET_MU(pollset) (&grpc_polling_mu)
#endif /* GRPC_INTERNAL_CORE_IOMGR_POLLSET_WINDOWS_H */

@ -478,8 +478,8 @@ done:
return allocated_port1 >= 0 ? allocated_port1 : allocated_port2;
}
int grpc_tcp_server_get_fd(grpc_tcp_server *s, unsigned index) {
return (index < s->nports) ? s->ports[index].fd : -1;
int grpc_tcp_server_get_fd(grpc_tcp_server *s, unsigned port_index) {
return (port_index < s->nports) ? s->ports[port_index].fd : -1;
}
void grpc_tcp_server_start(grpc_exec_ctx *exec_ctx, grpc_tcp_server *s,

@ -278,7 +278,7 @@ static void on_read(grpc_exec_ctx *exec_ctx, void *arg, int success) {
/* Tell the registered callback that data is available to read. */
GPR_ASSERT(sp->read_cb);
sp->read_cb(sp->fd, sp->server->grpc_server);
sp->read_cb(sp->emfd, sp->server->grpc_server);
/* Re-arm the notification event so we get another chance to read. */
grpc_fd_notify_on_read(exec_ctx, sp->emfd, &sp->read_closure);
@ -399,8 +399,8 @@ done:
return allocated_port1 >= 0 ? allocated_port1 : allocated_port2;
}
int grpc_udp_server_get_fd(grpc_udp_server *s, unsigned index) {
return (index < s->nports) ? s->ports[index].fd : -1;
int grpc_udp_server_get_fd(grpc_udp_server *s, unsigned port_index) {
return (port_index < s->nports) ? s->ports[port_index].fd : -1;
}
void grpc_udp_server_start(grpc_exec_ctx *exec_ctx, grpc_udp_server *s,

@ -43,7 +43,7 @@ typedef struct grpc_server grpc_server;
typedef struct grpc_udp_server grpc_udp_server;
/* Called when data is available to read from the socket. */
typedef void (*grpc_udp_server_read_cb)(int fd, grpc_server *server);
typedef void (*grpc_udp_server_read_cb)(grpc_fd *emfd, grpc_server *server);
/* Create a server, initially not bound to any ports */
grpc_udp_server *grpc_udp_server_create(void);

@ -212,7 +212,7 @@ double gpr_histogram_percentile(gpr_histogram *h, double percentile) {
}
double gpr_histogram_mean(gpr_histogram *h) {
GPR_ASSERT(h->count);
GPR_ASSERT(h->count != 0);
return h->sum / h->count;
}

@ -52,11 +52,11 @@ static struct timespec timespec_from_gpr(gpr_timespec gts) {
#if _POSIX_TIMERS > 0
static gpr_timespec gpr_from_timespec(struct timespec ts,
gpr_clock_type clock) {
gpr_clock_type clock_type) {
gpr_timespec rv;
rv.tv_sec = ts.tv_sec;
rv.tv_nsec = (int)ts.tv_nsec;
rv.clock_type = clock;
rv.clock_type = clock_type;
return rv;
}
@ -65,16 +65,16 @@ static clockid_t clockid_for_gpr_clock[] = {CLOCK_MONOTONIC, CLOCK_REALTIME};
void gpr_time_init(void) {}
gpr_timespec gpr_now(gpr_clock_type clock) {
gpr_timespec gpr_now(gpr_clock_type clock_type) {
struct timespec now;
GPR_ASSERT(clock != GPR_TIMESPAN);
if (clock == GPR_CLOCK_PRECISE) {
GPR_ASSERT(clock_type != GPR_TIMESPAN);
if (clock_type == GPR_CLOCK_PRECISE) {
gpr_timespec ret;
gpr_precise_clock_now(&ret);
return ret;
} else {
clock_gettime(clockid_for_gpr_clock[clock], &now);
return gpr_from_timespec(now, clock);
clock_gettime(clockid_for_gpr_clock[clock_type], &now);
return gpr_from_timespec(now, clock_type);
}
}
#else

@ -193,15 +193,15 @@ void grpc_chttp2_hptbl_destroy(grpc_chttp2_hptbl *tbl) {
}
grpc_mdelem *grpc_chttp2_hptbl_lookup(const grpc_chttp2_hptbl *tbl,
gpr_uint32 index) {
gpr_uint32 tbl_index) {
/* Static table comes first, just return an entry from it */
if (index <= GRPC_CHTTP2_LAST_STATIC_ENTRY) {
return tbl->static_ents[index - 1];
if (tbl_index <= GRPC_CHTTP2_LAST_STATIC_ENTRY) {
return tbl->static_ents[tbl_index - 1];
}
/* Otherwise, find the value in the list of valid entries */
index -= (GRPC_CHTTP2_LAST_STATIC_ENTRY + 1);
if (index < tbl->num_ents) {
gpr_uint32 offset = (tbl->num_ents - 1u - index + tbl->first_ent) %
tbl_index -= (GRPC_CHTTP2_LAST_STATIC_ENTRY + 1);
if (tbl_index < tbl->num_ents) {
gpr_uint32 offset = (tbl->num_ents - 1u - tbl_index + tbl->first_ent) %
GRPC_CHTTP2_MAX_TABLE_COUNT;
return tbl->ents[offset];
}

@ -274,10 +274,11 @@ static grpc_mdelem *add_elem(grpc_chttp2_hpack_compressor *c,
return elem_to_unref;
}
static void emit_indexed(grpc_chttp2_hpack_compressor *c, gpr_uint32 index,
static void emit_indexed(grpc_chttp2_hpack_compressor *c, gpr_uint32 elem_index,
framer_state *st) {
gpr_uint32 len = GRPC_CHTTP2_VARINT_LENGTH(index, 1);
GRPC_CHTTP2_WRITE_VARINT(index, 1, 0x80, add_tiny_header_data(st, len), len);
gpr_uint32 len = GRPC_CHTTP2_VARINT_LENGTH(elem_index, 1);
GRPC_CHTTP2_WRITE_VARINT(elem_index, 1, 0x80, add_tiny_header_data(st, len),
len);
}
static gpr_slice get_wire_value(grpc_mdelem *elem, gpr_uint8 *huffman_prefix) {
@ -363,9 +364,10 @@ static void emit_lithdr_noidx_v(grpc_chttp2_hpack_compressor *c,
add_header_data(st, gpr_slice_ref(value_slice));
}
static gpr_uint32 dynidx(grpc_chttp2_hpack_compressor *c, gpr_uint32 index) {
static gpr_uint32 dynidx(grpc_chttp2_hpack_compressor *c,
gpr_uint32 elem_index) {
return 1 + GRPC_CHTTP2_LAST_STATIC_ENTRY + c->tail_remote_index +
c->table_elems - index;
c->table_elems - elem_index;
}
/* encode an mdelem; returns metadata element to unref */

@ -274,14 +274,14 @@ void grpc_metadata_batch_link_tail(grpc_metadata_batch *batch,
}
void grpc_metadata_batch_merge(grpc_metadata_batch *target,
grpc_metadata_batch *add) {
grpc_metadata_batch *to_add) {
grpc_linked_mdelem *l;
grpc_linked_mdelem *next;
for (l = add->list.head; l; l = next) {
for (l = to_add->list.head; l; l = next) {
next = l->next;
link_tail(&target->list, l);
}
for (l = add->garbage.head; l; l = next) {
for (l = to_add->garbage.head; l; l = next) {
next = l->next;
link_tail(&target->garbage, l);
}

@ -154,10 +154,10 @@ void MetadataCredentialsPluginWrapper::Destroy(void* wrapper) {
void MetadataCredentialsPluginWrapper::GetMetadata(
void* wrapper, const char* service_url,
grpc_credentials_plugin_metadata_cb cb, void* user_data) {
GPR_ASSERT(wrapper != nullptr);
GPR_ASSERT(wrapper);
MetadataCredentialsPluginWrapper* w =
reinterpret_cast<MetadataCredentialsPluginWrapper*>(wrapper);
if (w->plugin_ == nullptr) {
if (!w->plugin_) {
cb(user_data, NULL, 0, GRPC_STATUS_OK, NULL);
return;
}
@ -177,11 +177,12 @@ void MetadataCredentialsPluginWrapper::InvokePlugin(
Status status = plugin_->GetMetadata(service_url, &metadata);
std::vector<grpc_metadata> md;
for (auto it = metadata.begin(); it != metadata.end(); ++it) {
md.push_back({it->first.c_str(),
it->second.data(),
it->second.size(),
0,
{{nullptr, nullptr, nullptr, nullptr}}});
grpc_metadata md_entry;
md_entry.key = it->first.c_str();
md_entry.value = it->second.data();
md_entry.value_length = it->second.size();
md_entry.flags = 0;
md.push_back(md_entry);
}
cb(user_data, md.empty() ? nullptr : &md[0], md.size(),
static_cast<grpc_status_code>(status.error_code()),

@ -51,7 +51,7 @@ void AuthMetadataProcessorAyncWrapper::Process(
void* wrapper, grpc_auth_context* context, const grpc_metadata* md,
size_t num_md, grpc_process_auth_metadata_done_cb cb, void* user_data) {
auto* w = reinterpret_cast<AuthMetadataProcessorAyncWrapper*>(wrapper);
if (w->processor_ == nullptr) {
if (!w->processor_) {
// Early exit.
cb(user_data, nullptr, 0, nullptr, 0, GRPC_STATUS_OK, nullptr);
return;
@ -84,20 +84,22 @@ void AuthMetadataProcessorAyncWrapper::InvokeProcessor(
std::vector<grpc_metadata> consumed_md;
for (auto it = consumed_metadata.begin(); it != consumed_metadata.end();
++it) {
consumed_md.push_back({it->first.c_str(),
it->second.data(),
it->second.size(),
0,
{{nullptr, nullptr, nullptr, nullptr}}});
grpc_metadata md_entry;
md_entry.key = it->first.c_str();
md_entry.value = it->second.data();
md_entry.value_length = it->second.size();
md_entry.flags = 0;
consumed_md.push_back(md_entry);
}
std::vector<grpc_metadata> response_md;
for (auto it = response_metadata.begin(); it != response_metadata.end();
++it) {
response_md.push_back({it->first.c_str(),
it->second.data(),
it->second.size(),
0,
{{nullptr, nullptr, nullptr, nullptr}}});
grpc_metadata md_entry;
md_entry.key = it->first.c_str();
md_entry.value = it->second.data();
md_entry.value_length = it->second.size();
md_entry.flags = 0;
response_md.push_back(md_entry);
}
auto consumed_md_data = consumed_md.empty() ? nullptr : &consumed_md[0];
auto response_md_data = response_md.empty() ? nullptr : &response_md[0];

@ -45,17 +45,21 @@ namespace Grpc.Core.Internal.Tests
[Test]
public void CreateAndDestroy()
{
GrpcEnvironment.AddRef();
var cq = CompletionQueueSafeHandle.Create();
cq.Dispose();
GrpcEnvironment.Release();
}
[Test]
public void CreateAndShutdown()
{
GrpcEnvironment.AddRef();
var cq = CompletionQueueSafeHandle.Create();
cq.Shutdown();
var ev = cq.Next();
cq.Dispose();
GrpcEnvironment.Release();
Assert.AreEqual(GRPCCompletionType.Shutdown, ev.type);
Assert.AreNotEqual(IntPtr.Zero, ev.success);
Assert.AreEqual(IntPtr.Zero, ev.tag);

@ -126,8 +126,6 @@ namespace Grpc.IntegrationTesting
new ChannelOption(ChannelOptions.SslTargetNameOverride, options.ServerHostOverride)
};
}
Console.WriteLine(options.ServerHost);
Console.WriteLine(options.ServerPort);
var channel = new Channel(options.ServerHost, options.ServerPort, credentials, channelOptions);
TestService.TestServiceClient client = new TestService.TestServiceClient(channel);
await RunTestCaseAsync(client, options);

@ -0,0 +1,6 @@
reporting:
watermarks:
statements: [80, 95]
lines: [80, 95]
functions: [80, 95]
branches: [80, 95]

@ -1,4 +1,7 @@
{
"variables" : {
'config': '<!(echo $CONFIG)'
},
"targets" : [
{
'include_dirs': [
@ -22,6 +25,18 @@
'pkg_config_grpc': '<!(pkg-config --exists grpc >/dev/null 2>&1 && echo true || echo false)'
},
'conditions': [
['config=="gcov"', {
'cflags': [
'-ftest-coverage',
'-fprofile-arcs',
'-O0'
],
'ldflags': [
'-ftest-coverage',
'-fprofile-arcs'
]
}
],
['pkg_config_grpc == "true"', {
'link_settings': {
'libraries': [

@ -21,8 +21,9 @@
},
"scripts": {
"lint": "node ./node_modules/jshint/bin/jshint src test examples interop index.js",
"test": "node ./node_modules/mocha/bin/mocha && npm run-script lint",
"gen_docs": "./node_modules/.bin/jsdoc -c jsdoc_conf.json"
"test": "./node_modules/.bin/mocha && npm run-script lint",
"gen_docs": "./node_modules/.bin/jsdoc -c jsdoc_conf.json",
"coverage": "./node_modules/.bin/istanbul cover ./node_modules/.bin/_mocha"
},
"dependencies": {
"bindings": "^1.2.0",
@ -33,6 +34,7 @@
"devDependencies": {
"async": "^0.9.0",
"google-auth-library": "^0.9.2",
"istanbul": "^0.3.21",
"jsdoc": "^3.3.2",
"jshint": "^2.5.0",
"minimist": "^1.1.0",

@ -89,7 +89,7 @@ static NSString * const kRemoteSSLHost = @"grpc-test.sandbox.google.com";
}
- (void)setUp {
_service = [[RMTTestService alloc] initWithHost:self.class.host];
_service = [RMTTestService serviceWithHost:self.class.host];
}
- (void)testEmptyUnaryRPC {
@ -274,17 +274,17 @@ static NSString * const kRemoteSSLHost = @"grpc-test.sandbox.google.com";
- (void)testCancelAfterFirstResponseRPC {
__weak XCTestExpectation *expectation = [self expectationWithDescription:@"CancelAfterFirstResponse"];
// A buffered pipe to which we write a single value but never close
GRXBufferedPipe *requestsBuffer = [[GRXBufferedPipe alloc] init];
__block BOOL receivedResponse = NO;
id request = [RMTStreamingOutputCallRequest messageWithPayloadSize:@21782
requestedResponseSize:@31415];
[requestsBuffer writeValue:request];
__block ProtoRPC *call =
[_service RPCToFullDuplexCallWithRequestsWriter:requestsBuffer
eventHandler:^(BOOL done,

@ -0,0 +1,60 @@
import types
import unittest
import pytest
class LoadTestsSuiteCollector(pytest.Collector):
def __init__(self, name, parent, suite):
super(LoadTestsSuiteCollector, self).__init__(name, parent=parent)
self.suite = suite
self.obj = suite
def collect(self):
collected = []
for case in self.suite:
if isinstance(case, unittest.TestCase):
collected.append(LoadTestsCase(case.id(), self, case))
elif isinstance(case, unittest.TestSuite):
collected.append(
LoadTestsSuiteCollector('suite_child_of_mine', self, case))
return collected
def reportinfo(self):
return str(self.suite)
class LoadTestsCase(pytest.Function):
def __init__(self, name, parent, item):
super(LoadTestsCase, self).__init__(name, parent, callobj=self._item_run)
self.item = item
def _item_run(self):
result = unittest.TestResult()
self.item(result)
if result.failures:
test_method, trace = result.failures[0]
pytest.fail(trace, False)
elif result.errors:
test_method, trace = result.errors[0]
pytest.fail(trace, False)
elif result.skipped:
test_method, reason = result.skipped[0]
pytest.skip(reason)
def pytest_pycollect_makeitem(collector, name, obj):
if name == 'load_tests' and isinstance(obj, types.FunctionType):
suite = unittest.TestSuite()
loader = unittest.TestLoader()
pattern = '*'
try:
# Check that the 'load_tests' object is actually a callable that actually
# accepts the arguments expected for the load_tests protocol.
suite = obj(loader, suite, pattern)
except Exception as e:
return None
else:
return LoadTestsSuiteCollector(name, collector, suite)

@ -3,5 +3,5 @@ forms a complete end-to-end test.
To add a new test or fixture:
- add the code to the relevant directory
- update gen_build_json.py to reflect the change
- update gen_build_yaml.py to reflect the change
- regenerate projects

@ -49,12 +49,12 @@ static grpc_pollset g_pollset;
static int g_number_of_reads = 0;
static int g_number_of_bytes_read = 0;
static void on_read(int fd, grpc_server *server) {
static void on_read(grpc_fd *emfd, grpc_server *server) {
char read_buffer[512];
ssize_t byte_count;
gpr_mu_lock(GRPC_POLLSET_MU(&g_pollset));
byte_count = recv(fd, read_buffer, sizeof(read_buffer), 0);
byte_count = recv(emfd->fd, read_buffer, sizeof(read_buffer), 0);
g_number_of_reads++;
g_number_of_bytes_read += (int)byte_count;

@ -35,6 +35,7 @@
#include <grpc/support/port_platform.h>
#include <grpc/support/log.h>
#include "src/core/support/string.h"
#include <stdlib.h>
#include <signal.h>
@ -88,25 +89,43 @@ static void install_crash_handler() {
#include <stdio.h>
#include <string.h>
#include <grpc/support/useful.h>
#include <errno.h>
static char g_alt_stack[MINSIGSTKSZ];
#define MAX_FRAMES 32
/* signal safe output */
static void output_string(const char *string) {
size_t len = strlen(string);
ssize_t r;
do {
r = write(STDERR_FILENO, string, len);
} while (r == -1 && errno == EINTR);
}
static void output_num(long num) {
char buf[GPR_LTOA_MIN_BUFSIZE];
gpr_ltoa(num, buf);
output_string(buf);
}
static void crash_handler(int signum, siginfo_t *info, void *data) {
void *addrlist[MAX_FRAMES + 1];
int addrlen;
int i;
char **symlist;
fprintf(stderr, "Caught signal %d\n", signum);
output_string("\n\n\n*******************************\nCaught signal ");
output_num(signum);
output_string("\n");
addrlen = backtrace(addrlist, GPR_ARRAY_SIZE(addrlist));
symlist = backtrace_symbols(addrlist, addrlen);
for (i = 0; i < addrlen; i++) {
fprintf(stderr, " %s\n", symlist[i]);
if (addrlen == 0) {
output_string(" no backtrace\n");
} else {
backtrace_symbols_fd(addrlist, addrlen, STDERR_FILENO);
}
free(symlist);
raise(signum);
}
@ -114,6 +133,7 @@ static void crash_handler(int signum, siginfo_t *info, void *data) {
static void install_crash_handler() {
stack_t ss;
struct sigaction sa;
memset(&ss, 0, sizeof(ss));
memset(&sa, 0, sizeof(sa));
ss.ss_size = sizeof(g_alt_stack);

File diff suppressed because it is too large Load Diff

@ -31,9 +31,9 @@
*
*/
#include <atomic>
#include <mutex>
#include <thread>
#include <time.h>
#include <grpc++/channel.h>
#include <grpc++/client_context.h>
@ -44,6 +44,7 @@
#include <grpc++/server_builder.h>
#include <grpc++/server_context.h>
#include <grpc/grpc.h>
#include <grpc/support/atm.h>
#include <grpc/support/thd.h>
#include <grpc/support/time.h>
#include <gtest/gtest.h>
@ -102,12 +103,17 @@ class TestServiceImpl : public ::grpc::cpp::test::util::TestService::Service {
public:
static void BidiStream_Sender(
ServerReaderWriter<EchoResponse, EchoRequest>* stream,
std::atomic<bool>* should_exit) {
gpr_atm* should_exit) {
EchoResponse response;
response.set_message(kLargeString);
while (!should_exit->load()) {
// TODO(vpai): Decide if the below requires blocking annotation
std::this_thread::sleep_for(std::chrono::milliseconds(1));
while (gpr_atm_acq_load(should_exit) == static_cast<gpr_atm>(0)) {
struct timespec tv = {0, 1000000}; // 1 ms
struct timespec rem;
// TODO (vpai): Mark this blocking
while (nanosleep(&tv, &rem) != 0) {
tv = rem;
};
stream->Write(response);
}
}
@ -117,15 +123,21 @@ class TestServiceImpl : public ::grpc::cpp::test::util::TestService::Service {
ServerReaderWriter<EchoResponse, EchoRequest>* stream)
GRPC_OVERRIDE {
EchoRequest request;
std::atomic<bool> should_exit(false);
gpr_atm should_exit;
gpr_atm_rel_store(&should_exit, static_cast<gpr_atm>(0));
std::thread sender(
std::bind(&TestServiceImpl::BidiStream_Sender, stream, &should_exit));
while (stream->Read(&request)) {
// TODO(vpai): Decide if the below requires blocking annotation
std::this_thread::sleep_for(std::chrono::milliseconds(3));
struct timespec tv = {0, 3000000}; // 3 ms
struct timespec rem;
// TODO (vpai): Mark this blocking
while (nanosleep(&tv, &rem) != 0) {
tv = rem;
};
}
should_exit.store(true);
gpr_atm_rel_store(&should_exit, static_cast<gpr_atm>(1));
sender.join();
return Status::OK;
}
@ -149,7 +161,7 @@ class End2endTest : public ::testing::Test {
void ResetStub() {
std::shared_ptr<Channel> channel =
CreateChannel(server_address_.str(), InsecureCredentials());
stub_ = std::move(grpc::cpp::test::util::TestService::NewStub(channel));
stub_ = grpc::cpp::test::util::TestService::NewStub(channel);
}
std::unique_ptr<grpc::cpp::test::util::TestService::Stub> stub_;

@ -0,0 +1,111 @@
#!/bin/bash
# Copyright 2015, Google Inc.
# All rights reserved.
#
# Redistribution and use in source and binary forms, with or without
# modification, are permitted provided that the following conditions are
# met:
#
# * Redistributions of source code must retain the above copyright
# notice, this list of conditions and the following disclaimer.
# * Redistributions in binary form must reproduce the above
# copyright notice, this list of conditions and the following disclaimer
# in the documentation and/or other materials provided with the
# distribution.
# * Neither the name of Google Inc. nor the names of its
# contributors may be used to endorse or promote products derived from
# this software without specific prior written permission.
#
# THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
# "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
# LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
# A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
# OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
# SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
# LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
# DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
# THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
# (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
# OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
#
# This script is invoked by run_interop_tests.py to accommodate
# "interop tests under docker" scenario. You should never need to call this
# script on your own.
set -ex
cd `dirname $0`/../..
git_root=`pwd`
cd -
mkdir -p /tmp/ccache
# Use image name based on Dockerfile checksum
DOCKER_IMAGE_NAME=grpc_jenkins_slave${docker_suffix}_`sha1sum tools/jenkins/grpc_jenkins_slave/Dockerfile | cut -f1 -d\ `
# Make sure docker image has been built. Should be instantaneous if so.
docker build -t $DOCKER_IMAGE_NAME tools/jenkins/grpc_jenkins_slave$docker_suffix
# Create a local branch so the child Docker script won't complain
git branch -f jenkins-docker
# Make sure the CID files are gone.
rm -f prepare.cid server.cid client.cid
# Prepare image for interop tests
docker run \
-e CCACHE_DIR=/tmp/ccache \
-i $TTY_FLAG \
-v "$git_root:/var/local/jenkins/grpc" \
-v /tmp/ccache:/tmp/ccache \
--cidfile=prepare.cid \
$DOCKER_IMAGE_NAME \
bash -l /var/local/jenkins/grpc/tools/jenkins/docker_prepare_interop_tests.sh || DOCKER_FAILED="true"
PREPARE_CID=`cat prepare.cid`
# Create image from the container, we will spawn one docker for clients
# and one for servers.
INTEROP_IMAGE=interop_`uuidgen`
docker commit $PREPARE_CID $INTEROP_IMAGE
# remove container, possibly killing it first
docker rm -f $PREPARE_CID || true
echo "Successfully built image $INTEROP_IMAGE"
# run interop servers under docker in the background
docker run \
-d -i \
$SERVERS_DOCKER_EXTRA_ARGS \
--cidfile=server.cid \
$INTEROP_IMAGE bash -l /var/local/git/grpc/tools/jenkins/docker_run_interop_servers.sh
SERVER_CID=`cat server.cid`
SERVER_PORTS=""
for tuple in $SERVER_PORT_TUPLES
do
# lookup under which port docker exposes given internal port
exposed_port=`docker port $SERVER_CID ${tuple#*:} | awk -F ":" '{print $NF}'`
# override the port for corresponding cloud_to_cloud server
SERVER_PORTS+=" --override_server ${tuple%:*}=localhost:$exposed_port"
echo "${tuple%:*} server is exposed under port $exposed_port"
done
# run interop clients
docker run \
-e "RUN_TESTS_COMMAND=$RUN_TESTS_COMMAND $SERVER_PORTS" \
-w /var/local/git/grpc \
-i $TTY_FLAG \
--net=host \
--cidfile=client.cid \
$INTEROP_IMAGE bash -l /var/local/git/grpc/tools/jenkins/docker_run_interop_tests.sh || DOCKER_FAILED="true"
CLIENT_CID=`cat client.cid`
echo "killing and removing server container $SERVER_CID"
docker rm -f $SERVER_CID || true
docker cp $CLIENT_CID:/var/local/git/grpc/report.xml $git_root
docker rm -f $CLIENT_CID || true
docker rmi -f $DOCKER_IMAGE_NAME || true

@ -0,0 +1,79 @@
#!/bin/bash
# Copyright 2015, Google Inc.
# All rights reserved.
#
# Redistribution and use in source and binary forms, with or without
# modification, are permitted provided that the following conditions are
# met:
#
# * Redistributions of source code must retain the above copyright
# notice, this list of conditions and the following disclaimer.
# * Redistributions in binary form must reproduce the above
# copyright notice, this list of conditions and the following disclaimer
# in the documentation and/or other materials provided with the
# distribution.
# * Neither the name of Google Inc. nor the names of its
# contributors may be used to endorse or promote products derived from
# this software without specific prior written permission.
#
# THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
# "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
# LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
# A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
# OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
# SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
# LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
# DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
# THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
# (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
# OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
#
# This script is invoked by run_jekins.sh. It contains the test logic
# that should run inside a docker container.
set -e
mkdir -p /var/local/git
git clone --recursive /var/local/jenkins/grpc /var/local/git/grpc
cd /var/local/git/grpc
nvm use 0.12
rvm use ruby-2.1
# TODO(jtattermusch): use cleaner way to install root certs
mkdir -p /usr/local/share/grpc
cp etc/roots.pem /usr/local/share/grpc/
# build C++ interop client & server
make interop_client interop_server
# build C# interop client & server
make install_grpc_csharp_ext
(cd src/csharp && mono /var/local/NuGet.exe restore Grpc.sln)
(cd src/csharp && xbuild Grpc.sln)
# build Node interop client & server
npm install -g node-gyp
make install_c -C /var/local/git/grpc
(cd src/node && npm install && node-gyp rebuild)
# build Ruby interop client and server
(cd src/ruby && gem update bundler && bundle && rake compile:grpc)
# TODO(jtattermusch): add python
# build PHP interop client
# TODO(jtattermusch): prerequisites for PHP should be installed sooner than here.
# Install composer
curl -sS https://getcomposer.org/installer | php
mv composer.phar /usr/local/bin/composer
# Download the patched PHP protobuf so that PHP gRPC clients can be generated
# from proto3 schemas.
git clone https://github.com/stanley-cheung/Protobuf-PHP.git /var/local/git/protobuf-php
(cd src/php/ext/grpc && phpize && ./configure && make)
rvm all do gem install ronn rake
(cd third_party/protobuf && make install)
(cd /var/local/git/protobuf-php \
&& rvm all do rake pear:package version=1.0 \
&& pear install Protobuf-1.0.tgz)
(cd src/php && composer install)
(cd src/php && protoc-gen-php -i tests/interop/ -o tests/interop/ tests/interop/test.proto)

@ -1,5 +1,4 @@
#!/bin/sh
#!/bin/bash
# Copyright 2015, Google Inc.
# All rights reserved.
#
@ -28,27 +27,24 @@
# THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
# (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
# OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
#
# This script is invoked by run_jekins.sh. It contains the test logic
# that should run inside a docker container.
set -e
language=$1
test_case=$2
cd /var/local/git/grpc
nvm use 0.12
rvm use ruby-2.1
set -e
if [ "$language" = "c++" ]
then
sudo docker run grpc/cxx /var/local/git/grpc/bins/opt/interop_client --enable_ssl --use_prod_roots --server_host_override=grpc-test.sandbox.google.com --server_host=grpc-test.sandbox.google.com --server_port=443 --test_case=$test_case
elif [ "$language" = "node" ]
then
sudo docker run grpc/node /usr/bin/nodejs /var/local/git/grpc/src/node/interop/interop_client.js --use_tls=true --use_test_ca=true --server_port=443 --server_host=grpc-test.sandbox.google.com --server_host_override=grpc-test.sandbox.google.com --test_case=$test_case
elif [ "$language" = "ruby" ]
then
cmd_prefix="SSL_CERT_FILE=/cacerts/roots.pem ruby /var/local/git/grpc/src/ruby/bin/interop/interop_client.rb --use_tls --server_port=443 --server_host=grpc-test.sandbox.google.com --server_host_override=grpc-test.sandbox.google.com "
cmd="$cmd_prefix --test_case=$test_case"
sudo docker run grpc/ruby bin/bash -l -c '$cmd'
elif [ "$language" = "php" ]
then
sudo docker run -e SSL_CERT_FILE=/cacerts/roots.pem grpc/php /var/local/git/grpc/src/php/bin/interop_client.sh --server_port=443 --server_host=grpc-test.sandbox.google.com --server_host_override=grpc-test.sandbox.google.com --test_case=$test_case
else
echo "interop testss not added for $language"
exit 1
fi
# If port env variable is set, run corresponding interop server on given port in background.
# TODO(jtattermusch): ideally, run_interop_tests.py would generate the commands to run servers.
[ -z "${SERVER_PORT_cxx}" ] || bins/opt/interop_server --enable_ssl --port=${SERVER_PORT_cxx} &
[ -z "${SERVER_PORT_node}" ] || node src/node/interop/interop_server.js --use_tls=true --port=${SERVER_PORT_node} &
[ -z "${SERVER_PORT_ruby}" ] || ruby src/ruby/bin/interop/interop_server.rb --use_tls --port=${SERVER_PORT_ruby} &
[ -z "${SERVER_PORT_csharp}" ] || (cd src/csharp/Grpc.IntegrationTesting.Server/bin/Debug && mono Grpc.IntegrationTesting.Server.exe --use_tls --port=${SERVER_PORT_csharp}) &
sleep infinity

@ -0,0 +1,39 @@
#!/bin/bash
# Copyright 2015, Google Inc.
# All rights reserved.
#
# Redistribution and use in source and binary forms, with or without
# modification, are permitted provided that the following conditions are
# met:
#
# * Redistributions of source code must retain the above copyright
# notice, this list of conditions and the following disclaimer.
# * Redistributions in binary form must reproduce the above
# copyright notice, this list of conditions and the following disclaimer
# in the documentation and/or other materials provided with the
# distribution.
# * Neither the name of Google Inc. nor the names of its
# contributors may be used to endorse or promote products derived from
# this software without specific prior written permission.
#
# THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
# "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
# LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
# A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
# OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
# SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
# LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
# DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
# THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
# (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
# OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
#
# This script is invoked by build_docker_and_run_interop_tests.sh inside
# a docker container. You should never need to call this script on your own.
set -e
nvm use 0.12
rvm use ruby-2.1
# run the cloud-to-prod interop tests
$RUN_TESTS_COMMAND

@ -28,8 +28,9 @@
# (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
# OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
#
# This script is invoked by build_docker_and_run_tests.py inside a docker
# This script is invoked by build_docker_and_run_tests.sh inside a docker
# container. You should never need to call this script on your own.
set -e
export CONFIG=$config

@ -32,7 +32,7 @@
FROM debian:jessie
# Install Git.
# Install Git and basic packages.
RUN apt-get update && apt-get install -y \
autoconf \
autotools-dev \
@ -44,6 +44,7 @@ RUN apt-get update && apt-get install -y \
gcc-multilib \
git \
gyp \
lcov \
libc6 \
libc6-dbg \
libc6-dev \

@ -32,7 +32,7 @@
FROM 32bit/debian:jessie
# Install Git.
# Install Git and basic packages.
RUN apt-get update && apt-get install -y \
autoconf \
autotools-dev \
@ -44,6 +44,7 @@ RUN apt-get update && apt-get install -y \
gcc-multilib \
git \
gyp \
lcov \
libc6 \
libc6-dbg \
libc6-dev \

@ -54,7 +54,7 @@ if [ "$platform" == "linux" ]; then
docker build -t $DOCKER_IMAGE_NAME tools/jenkins/grpc_linuxbrew
# run per-language homebrew installation script
docker run $DOCKER_IMAGE_NAME bash -l \
docker run --rm=true $DOCKER_IMAGE_NAME bash -l \
-c "nvm use 0.12; \
npm set unsafe-perm true; \
rvm use ruby-2.1; \

@ -56,9 +56,6 @@ then
./tools/run_tests/run_tests.py --use_docker -t -l $language -c $config -x report.xml $@ || true
elif [ "$platform" == "interop" ]
then
python tools/run_tests/run_interops.py --language=$language $@
elif [ "$platform" == "windows" ]
then
echo "building $language on Windows"
@ -83,6 +80,12 @@ then
echo "building $language on FreeBSD"
MAKE=gmake ./tools/run_tests/run_tests.py -t -l $language -c $config -x report.xml $@ || true
elif [ "$platform" == "interop" ]
then
echo "building interop tests for language $language"
./tools/run_tests/run_interop_tests.py --use_docker -t -l $language --cloud_to_prod --server all || true
else
echo "Unknown platform $platform"
exit 1

@ -0,0 +1,330 @@
#!/usr/bin/env python
# Copyright 2015, Google Inc.
# All rights reserved.
#
# Redistribution and use in source and binary forms, with or without
# modification, are permitted provided that the following conditions are
# met:
#
# * Redistributions of source code must retain the above copyright
# notice, this list of conditions and the following disclaimer.
# * Redistributions in binary form must reproduce the above
# copyright notice, this list of conditions and the following disclaimer
# in the documentation and/or other materials provided with the
# distribution.
# * Neither the name of Google Inc. nor the names of its
# contributors may be used to endorse or promote products derived from
# this software without specific prior written permission.
#
# THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
# "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
# LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
# A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
# OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
# SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
# LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
# DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
# THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
# (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
# OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
"""Run interop (cross-language) tests in parallel."""
import argparse
import itertools
import xml.etree.cElementTree as ET
import jobset
import os
import subprocess
import sys
import time
_CLOUD_TO_PROD_BASE_ARGS = [
'--server_host_override=grpc-test.sandbox.google.com',
'--server_host=grpc-test.sandbox.google.com',
'--server_port=443']
_CLOUD_TO_CLOUD_BASE_ARGS = [
'--server_host_override=foo.test.google.fr']
# TOOD(jtattermusch) wrapped languages use this variable for location
# of roots.pem. We might want to use GRPC_DEFAULT_SSL_ROOTS_FILE_PATH
# supported by C core SslCredentials instead.
_SSL_CERT_ENV = { 'SSL_CERT_FILE':'/usr/local/share/grpc/roots.pem' }
# TODO(jtatttermusch) unify usage of --enable_ssl, --use_tls and --use_tls=true
class CXXLanguage:
def __init__(self):
self.client_cmdline_base = ['bins/opt/interop_client']
self.client_cwd = None
def cloud_to_prod_args(self):
return (self.client_cmdline_base + _CLOUD_TO_PROD_BASE_ARGS +
['--enable_ssl','--use_prod_roots'])
def cloud_to_cloud_args(self):
return (self.client_cmdline_base + _CLOUD_TO_CLOUD_BASE_ARGS +
['--enable_ssl'])
def cloud_to_prod_env(self):
return None
def __str__(self):
return 'c++'
class CSharpLanguage:
def __init__(self):
self.client_cmdline_base = ['mono', 'Grpc.IntegrationTesting.Client.exe']
self.client_cwd = 'src/csharp/Grpc.IntegrationTesting.Client/bin/Debug'
def cloud_to_prod_args(self):
return (self.client_cmdline_base + _CLOUD_TO_PROD_BASE_ARGS +
['--use_tls'])
def cloud_to_cloud_args(self):
return (self.client_cmdline_base + _CLOUD_TO_CLOUD_BASE_ARGS +
['--use_tls', '--use_test_ca'])
def cloud_to_prod_env(self):
return _SSL_CERT_ENV
def __str__(self):
return 'csharp'
class NodeLanguage:
def __init__(self):
self.client_cmdline_base = ['node', 'src/node/interop/interop_client.js']
self.client_cwd = None
def cloud_to_prod_args(self):
return (self.client_cmdline_base + _CLOUD_TO_PROD_BASE_ARGS +
['--use_tls=true'])
def cloud_to_cloud_args(self):
return (self.client_cmdline_base + _CLOUD_TO_CLOUD_BASE_ARGS +
['--use_tls=true', '--use_test_ca=true'])
def cloud_to_prod_env(self):
return _SSL_CERT_ENV
def __str__(self):
return 'node'
class PHPLanguage:
def __init__(self):
self.client_cmdline_base = ['src/php/bin/interop_client.sh']
self.client_cwd = None
def cloud_to_prod_args(self):
return (self.client_cmdline_base + _CLOUD_TO_PROD_BASE_ARGS +
['--use_tls'])
def cloud_to_cloud_args(self):
return (self.client_cmdline_base + _CLOUD_TO_CLOUD_BASE_ARGS +
['--use_tls', '--use_test_ca'])
def cloud_to_prod_env(self):
return _SSL_CERT_ENV
def __str__(self):
return 'php'
class RubyLanguage:
def __init__(self):
self.client_cmdline_base = ['ruby', 'src/ruby/bin/interop/interop_client.rb']
self.client_cwd = None
def cloud_to_prod_args(self):
return (self.client_cmdline_base + _CLOUD_TO_PROD_BASE_ARGS +
['--use_tls'])
def cloud_to_cloud_args(self):
return (self.client_cmdline_base + _CLOUD_TO_CLOUD_BASE_ARGS +
['--use_tls', '--use_test_ca'])
def cloud_to_prod_env(self):
return _SSL_CERT_ENV
def __str__(self):
return 'ruby'
# TODO(jtattermusch): add php and python once we get them working
_LANGUAGES = {
'c++' : CXXLanguage(),
'csharp' : CSharpLanguage(),
'node' : NodeLanguage(),
'php' : PHPLanguage(),
'ruby' : RubyLanguage(),
}
# languages supported as cloud_to_cloud servers
# TODO(jtattermusch): enable other languages as servers as well
_SERVERS = { 'c++' : 8010, 'node' : 8040, 'csharp': 8070 }
# TODO(jtattermusch): add empty_stream once C++ start supporting it.
# TODO(jtattermusch): add support for auth tests.
_TEST_CASES = ['large_unary', 'empty_unary', 'ping_pong',
'client_streaming', 'server_streaming',
'cancel_after_begin', 'cancel_after_first_response',
'timeout_on_sleeping_server']
def cloud_to_prod_jobspec(language, test_case):
"""Creates jobspec for cloud-to-prod interop test"""
cmdline = language.cloud_to_prod_args() + ['--test_case=%s' % test_case]
test_job = jobset.JobSpec(
cmdline=cmdline,
cwd=language.client_cwd,
shortname="cloud_to_prod:%s:%s" % (language, test_case),
environ=language.cloud_to_prod_env(),
timeout_seconds=60)
return test_job
def cloud_to_cloud_jobspec(language, test_case, server_name, server_host,
server_port):
"""Creates jobspec for cloud-to-cloud interop test"""
cmdline = language.cloud_to_cloud_args() + ['--test_case=%s' % test_case,
'--server_host=%s' % server_host,
'--server_port=%s' % server_port ]
test_job = jobset.JobSpec(
cmdline=cmdline,
cwd=language.client_cwd,
shortname="cloud_to_cloud:%s:%s_server:%s" % (language, server_name,
test_case),
timeout_seconds=60)
return test_job
argp = argparse.ArgumentParser(description='Run interop tests.')
argp.add_argument('-l', '--language',
choices=['all'] + sorted(_LANGUAGES),
nargs='+',
default=['all'],
help='Clients to run.')
argp.add_argument('-j', '--jobs', default=24, type=int)
argp.add_argument('--cloud_to_prod',
default=False,
action='store_const',
const=True,
help='Run cloud_to_prod tests.')
argp.add_argument('-s', '--server',
choices=['all'] + sorted(_SERVERS),
action='append',
help='Run cloud_to_cloud servers in a separate docker ' +
'image. Servers can only be started automatically if ' +
'--use_docker option is enabled.',
default=[])
argp.add_argument('--override_server',
action='append',
type=lambda kv: kv.split("="),
help='Use servername=HOST:PORT to explicitly specify a server. E.g. csharp=localhost:50000',
default=[])
argp.add_argument('-t', '--travis',
default=False,
action='store_const',
const=True)
argp.add_argument('--use_docker',
default=False,
action='store_const',
const=True,
help='Run all the interop tests under docker. That provides ' +
'additional isolation and prevents the need to install ' +
'language specific prerequisites. Only available on Linux.')
args = argp.parse_args()
servers = set(s for s in itertools.chain.from_iterable(_SERVERS.iterkeys()
if x == 'all' else [x]
for x in args.server))
if args.use_docker:
if not args.travis:
print 'Seen --use_docker flag, will run interop tests under docker.'
print
print 'IMPORTANT: The changes you are testing need to be locally committed'
print 'because only the committed changes in the current branch will be'
print 'copied to the docker environment.'
time.sleep(5)
child_argv = [ arg for arg in sys.argv if not arg == '--use_docker' ]
run_tests_cmd = ('tools/run_tests/run_interop_tests.py %s' %
" ".join(child_argv[1:]))
# cmdline args to pass to the container running servers.
servers_extra_docker_args = ''
server_port_tuples = ''
for server in servers:
port = _SERVERS[server]
servers_extra_docker_args += ' -p %s' % port
servers_extra_docker_args += ' -e SERVER_PORT_%s=%s' % (server.replace("+", "x"), port)
server_port_tuples += ' %s:%s' % (server, port)
env = os.environ.copy()
env['RUN_TESTS_COMMAND'] = run_tests_cmd
env['SERVERS_DOCKER_EXTRA_ARGS'] = servers_extra_docker_args
env['SERVER_PORT_TUPLES'] = server_port_tuples
if not args.travis:
env['TTY_FLAG'] = '-t' # enables Ctrl-C when not on Jenkins.
subprocess.check_call(['tools/jenkins/build_docker_and_run_interop_tests.sh'],
shell=True,
env=env)
sys.exit(0)
languages = set(_LANGUAGES[l]
for l in itertools.chain.from_iterable(
_LANGUAGES.iterkeys() if x == 'all' else [x]
for x in args.language))
jobs = []
if args.cloud_to_prod:
for language in languages:
for test_case in _TEST_CASES:
test_job = cloud_to_prod_jobspec(language, test_case)
jobs.append(test_job)
# default servers to "localhost" and the default port
server_addresses = dict((s, ("localhost", _SERVERS[s])) for s in servers)
for server in args.override_server:
server_name = server[0]
(server_host, server_port) = server[1].split(":")
server_addresses[server_name] = (server_host, server_port)
for server_name, server_address in server_addresses.iteritems():
(server_host, server_port) = server_address
for language in languages:
for test_case in _TEST_CASES:
test_job = cloud_to_cloud_jobspec(language,
test_case,
server_name,
server_host,
server_port)
jobs.append(test_job)
if not jobs:
print "No jobs to run."
sys.exit(1)
root = ET.Element('testsuites')
testsuite = ET.SubElement(root, 'testsuite', id='1', package='grpc', name='tests')
if jobset.run(jobs, newline_on_success=True, maxjobs=args.jobs, xml_report=testsuite):
jobset.message('SUCCESS', 'All tests passed', do_newline=True)
else:
jobset.message('FAILED', 'Some tests failed', do_newline=True)
tree = ET.ElementTree(root)
tree.write('report.xml', encoding='UTF-8')

@ -1,37 +0,0 @@
import argparse
import xml.etree.cElementTree as ET
import jobset
argp = argparse.ArgumentParser(description='Run interop tests.')
argp.add_argument('-l', '--language',
default='c++')
args = argp.parse_args()
# build job
build_job = jobset.JobSpec(cmdline=['tools/run_tests/run_interops_build.sh', '%s' % args.language],
shortname='build',
timeout_seconds=30*60)
# test jobs, each test is a separate job to run in parallel
_TESTS = ['large_unary', 'empty_unary', 'ping_pong', 'client_streaming', 'server_streaming']
jobs = []
jobNumber = 0
for test in _TESTS:
test_job = jobset.JobSpec(
cmdline=['tools/run_tests/run_interops_test.sh', '%s' % args.language, '%s' % test],
shortname=test,
timeout_seconds=15*60)
jobs.append(test_job)
jobNumber+=1
root = ET.Element('testsuites')
testsuite = ET.SubElement(root, 'testsuite', id='1', package='grpc', name='tests')
# always do the build of docker first, and then all the tests can run in parallel
jobset.run([build_job], maxjobs=1, xml_report=testsuite)
jobset.run(jobs, maxjobs=jobNumber, xml_report=testsuite)
tree = ET.ElementTree(root)
tree.write('report.xml', encoding='UTF-8')

@ -1,75 +0,0 @@
#!/bin/sh
# Copyright 2015, Google Inc.
# All rights reserved.
#
# Redistribution and use in source and binary forms, with or without
# modification, are permitted provided that the following conditions are
# met:
#
# * Redistributions of source code must retain the above copyright
# notice, this list of conditions and the following disclaimer.
# * Redistributions in binary form must reproduce the above
# copyright notice, this list of conditions and the following disclaimer
# in the documentation and/or other materials provided with the
# distribution.
# * Neither the name of Google Inc. nor the names of its
# contributors may be used to endorse or promote products derived from
# this software without specific prior written permission.
#
# THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
# "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
# LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
# A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
# OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
# SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
# LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
# DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
# THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
# (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
# OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
language=$1
set -e
#clean up any old docker files and start mirroring repository if not started already
sudo docker rmi -f grpc/cxx || true
sudo docker rmi -f grpc/base || true
sudo docker rmi -f 0.0.0.0:5000/grpc/base || true
sudo docker run -d -e GCS_BUCKET=docker-interop-images -e STORAGE_PATH=/admin/docker_images -p 5000:5000 google/docker-registry || true
#prepare building by pulling down base images and necessary files
sudo docker pull 0.0.0.0:5000/grpc/base
sudo docker tag -f 0.0.0.0:5000/grpc/base grpc/base
if [ "$language" = "c++" ]
then
gsutil cp -R gs://docker-interop-images/admin/service_account tools/dockerfile/grpc_cxx
gsutil cp -R gs://docker-interop-images/admin/cacerts tools/dockerfile/grpc_cxx
sudo docker build --no-cache -t grpc/cxx tools/dockerfile/grpc_cxx
elif [ "$language" = "node" ]
then
sudo docker pull 0.0.0.0:5000/grpc/node_base
sudo docker tag -f 0.0.0.0:5000/grpc/node_base grpc/node_base
gsutil cp -R gs://docker-interop-images/admin/service_account tools/dockerfile/grpc_node
gsutil cp -R gs://docker-interop-images/admin/cacerts tools/dockerfile/grpc_node
sudo docker build --no-cache -t grpc/node tools/dockerfile/grpc_node
elif [ "$language" = "ruby" ]
then
sudo docker pull 0.0.0.0:5000/grpc/ruby_base
sudo docker tag -f 0.0.0.0:5000/grpc/ruby_base grpc/ruby_base
gsutil cp -R gs://docker-interop-images/admin/service_account tools/dockerfile/grpc_ruby
gsutil cp -R gs://docker-interop-images/admin/cacerts tools/dockerfile/grpc_ruby
sudo docker build --no-cache -t grpc/ruby tools/dockerfile/grpc_ruby
elif [ "$language" = "php" ]
then
sudo docker pull 0.0.0.0:5000/grpc/php_base
sudo docker tag -f 0.0.0.0:5000/grpc/php_base grpc/php_base
gsutil cp -R gs://docker-interop-images/admin/service_account tools/dockerfile/grpc_php
gsutil cp -R gs://docker-interop-images/admin/cacerts tools/dockerfile/grpc_php
sudo docker build --no-cache -t grpc/php tools/dockerfile/grpc_php
else
echo "interop testss not added for $language"
exit 1
fi

@ -37,6 +37,19 @@ cd $(dirname $0)/../..
root=`pwd`
cd $root/src/node
export LD_LIBRARY_PATH=$root/libs/$CONFIG
$root/src/node/node_modules/mocha/bin/mocha --timeout 8000 $root/src/node/test
if [ "$CONFIG" = "gcov" ]
then
./node_modules/.bin/istanbul cover ./node_modules/.bin/_mocha -- \
--timeout 8000
cd build
gcov Release/obj.target/grpc/ext/*.o
lcov --base-directory . --directory . -c -o coverage.info
genhtml -o ../ext_coverage --num-spaces 2 -t 'Node gRPC test coverage' \
coverage.info
else
./node_modules/mocha/bin/mocha --timeout 8000
fi

@ -40,14 +40,4 @@ export DYLD_LIBRARY_PATH=$ROOT/libs/$CONFIG
export PATH=$ROOT/bins/$CONFIG:$ROOT/bins/$CONFIG/protobuf:$PATH
source "python"$PYVER"_virtual_environment"/bin/activate
# TODO(atash): These tests don't currently run under py.test and thus don't
# appear under the coverage report. Find a way to get these tests to work with
# py.test (or find another tool or *something*) that's acceptable to the rest of
# the team...
"python"$PYVER -m grpc_test._core_over_links_base_interface_test
"python"$PYVER -m grpc_test._crust_over_core_over_links_face_interface_test
"python"$PYVER -m grpc_test.beta._face_interface_test
"python"$PYVER -m grpc_test.framework._crust_over_core_face_interface_test
"python"$PYVER -m grpc_test.framework.core._base_interface_test
"python"$PYVER $GRPCIO_TEST/setup.py test -a "-n8 --cov=grpc --junitxml=./report.xml --timeout=300"

@ -557,7 +557,7 @@ argp.add_argument('--use_docker',
action='store_const',
const=True,
help="Run all the tests under docker. That provides " +
"additional isolation and prevents the need to installs " +
"additional isolation and prevents the need to install " +
"language specific prerequisites. Only available on Linux.")
argp.add_argument('--allow_flakes',
default=False,

@ -0,0 +1,35 @@
Zlib Native Nuget package
-------------------------
Uses [CoApp](http://coapp.org/) project to build the zlib package.
Prerequisites
-------------
Multiple versions of VS installed to be able to build all the targets:
* Visual Studio 2013
* Visual Studio 2010 (you might need SP1 to prevent LNK1123 error)
CoApp toolkit: http://downloads.coapp.org/files/CoApp.Tools.Powershell.msi
More details on installation: http://coapp.org/tutorials/installation.html
Building
--------
Build all flavors of zlib library using the provided batch file.
```
buildall.bat
```
Then, create NuGet package using powershell (you'll need the CoApp toolkit installed):
```
[THIS_DIRECTORY]> Write-NuGetPackage grpc.dependencies.zlib.autopkg
```
This will create three NuGet packages:
* the main dev package
* the redistributable package that contains just the binaries and no headers
* the symbols package (debug symbols)
Later, you can push the package to NuGet.org repo.
Attention: before pusing the resulting nuget package to public nuget repo, you have to be 100% sure it works correctly - there’s no way how to delete or update an already existing package.

@ -0,0 +1,51 @@
@echo off
setlocal
setlocal
call "C:\Program Files (x86)\Microsoft Visual Studio 12.0\VC\vcvarsall.bat" amd64
call :build x64 Release v120 || goto :eof
call :build x64 Debug v120 || goto :eof
endlocal
setlocal
call "C:\Program Files (x86)\Microsoft Visual Studio 12.0\VC\vcvarsall.bat" x86
call :build Win32 Release v120 || goto :eof
call :build Win32 Debug v120 || goto :eof
endlocal
REM setlocal
REM call "C:\Program Files (x86)\Microsoft Visual Studio 11.0\VC\vcvarsall.bat" amd64
REM call :build x64 Release v110 || goto :eof
REM call :build x64 Debug v110 || goto :eof
REM endlocal
REM setlocal
REM call "C:\Program Files (x86)\Microsoft Visual Studio 11.0\VC\vcvarsall.bat" x86
REM call :build Win32 Release v110 || goto :eof
REM call :build Win32 Debug v110 || goto :eof
REM endlocal
REM setlocal
REM call "C:\Program Files (x86)\Microsoft Visual Studio 10.0\VC\vcvarsall.bat" amd64
REM call :build x64 Release v100 || goto :eof
REM call :build x64 Debug v100 || goto :eof
REM endlocal
setlocal
call "C:\Program Files (x86)\Microsoft Visual Studio 10.0\VC\vcvarsall.bat" x86
call :build Win32 Release v100 || goto :eof
call :build Win32 Debug v100 || goto :eof
endlocal
goto :eof
:build
msbuild /P:Platform=%1 /P:Configuration=%2 /P:PlatformToolset=%3 /P:UsesConfigurationType=Dynamic /P:CallingConvention=cdecl .\zlib.sln || goto :eof
msbuild /P:Platform=%1 /P:Configuration=%2 /P:PlatformToolset=%3 /P:UsesConfigurationType=Dynamic /P:CallingConvention=stdcall .\zlib.sln || goto :eof
msbuild /P:Platform=%1 /P:Configuration=%2 /P:PlatformToolset=%3 /P:UsesConfigurationType=Static /P:CallingConvention=cdecl .\zlib.sln || goto :eof
msbuild /P:Platform=%1 /P:Configuration=%2 /P:PlatformToolset=%3 /P:UsesConfigurationType=Static /P:CallingConvention=stdcall .\zlib.sln || goto :eof
msbuild /P:Platform=%1 /P:Configuration=%2 /P:PlatformToolset=%3 /P:UsesConfigurationType=ltcg /P:CallingConvention=cdecl .\zlib.sln || goto :eof
msbuild /P:Platform=%1 /P:Configuration=%2 /P:PlatformToolset=%3 /P:UsesConfigurationType=ltcg /P:CallingConvention=stdcall .\zlib.sln || goto :eof
goto :eof

@ -0,0 +1,102 @@
@import @"version.inc";
configurations {
};
#define {
package-id = "grpc.dependencies.zlib";
}
nuget {
// the nuspec file metadata. Gets created/updated on build
nuspec {
id = ${package-id};
version : ${package-version};
title: gRPC Native Dependency: ZLib compression library;
authors: {Jean-loup Gailly, Mark Adler, Garrett Serack, Tim Rogers};
owners: {Jan Tattermusch};
licenseUrl: "http://zlib.net/zlib-license.html";
projectUrl: "http://github.com/jtattermusch/zlib";
iconUrl: "http://zlib.net/images/zlib3d-b1.png";
requireLicenseAcceptance:false;
summary:A zlib library;
description: @"A native zlib library.
zlib homepage: http://zlib.net";
releaseNotes: "Release of zlib 1.2.8 libraries.";
copyright: Copyright 2013;
tags: { zlib, native, CoApp };
};
// the files that go into the content folders
// (inserted into the nuspec file)
files {
// .targets file that are applied when redist package is installed from a managed project.
managed_build: {
#output {
package = redist;
};
#destination = "\build\portable-net45";
"managed_targets\${package-id}.redist.props";
"managed_targets\${package-id}.redist.targets";
};
include: { ..\..\..\third_party\zlib\zlib.h, ..\..\..\third_party\zlib\zconf.h };
docs: { ..\..\..\third_party\zlib\doc\**\* };
source += {
"..\..\..\third_party\zlib\adler32.c",
"..\..\..\third_party\zlib\compress.c",
"..\..\..\third_party\zlib\crc32.c",
"..\..\..\third_party\zlib\deflate.c",
"..\..\..\third_party\zlib\gzclose.c",
"..\..\..\third_party\zlib\gzlib.c",
"..\..\..\third_party\zlib\gzread.c",
"..\..\..\third_party\zlib\gzwrite.c",
"..\..\..\third_party\zlib\infback.c",
"..\..\..\third_party\zlib\inffast.c",
"..\..\..\third_party\zlib\inflate.c",
"..\..\..\third_party\zlib\inftrees.c",
"..\..\..\third_party\zlib\trees.c",
"..\..\..\third_party\zlib\uncompr.c",
"..\..\..\third_party\zlib\zutil.c",
"..\..\..\third_party\zlib\crc32.h",
"..\..\..\third_party\zlib\deflate.h",
"..\..\..\third_party\zlib\gzguts.h",
"..\..\..\third_party\zlib\inffast.h",
"..\..\..\third_party\zlib\inffixed.h",
"..\..\..\third_party\zlib\inflate.h",
"..\..\..\third_party\zlib\inftrees.h",
"..\..\..\third_party\zlib\trees.h",
"..\..\..\third_party\zlib\zconf.h",
"..\..\..\third_party\zlib\zlib.h",
"..\..\..\third_party\zlib\zutil.h",
"..\..\..\third_party\zlib\contrib\masmx64\inffas8664.c",
};
("v100,v120", "Win32,x64", "Release,Debug", "Dynamic", "cdecl,stdcall", "MultiByte") => {
[${0},${1},${2},${3},${4}] {
lib: { .\output\${0}\${1}\${2}\${3}\${4}\${5}\*.lib };
bin: { .\output\${0}\${1}\${2}\${3}\${4}\${5}\*.dll };
symbols: { .\output\${0}\${1}\${2}\${3}\${4}\${5}\*.pdb };
};
};
("v100,v120", "Win32,x64", "Release,Debug", "Static,ltcg", "cdecl,stdcall", "MultiByte") => {
[${0},${1},${2},${3},${4}] {
lib: { .\output\${0}\${1}\${2}\${3}\${4}\${5}\*.lib };
};
};
};
// the VC++ .targets file that gets generated and inserted into the ${d_content} folder
targets {
Defines += HAS_ZLIB;
[dynamic]
Defines += ZLIB_DLL;
[stdcall]
Defines += ZLIB_WINAPI;
};
}

@ -0,0 +1,13 @@
<?xml version="1.0" encoding="utf-8"?>
<Project ToolsVersion="4.0" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
<PropertyGroup>
<!-- Whether or not copy native dependencies to output directory after building -->
<CopyNativeDependencies Condition=" '$(CopyNativeDependencies)' == '' ">true</CopyNativeDependencies>
<!-- Set defaults for native dependencies if not already set. Properties can be overriden in the project files. -->
<NativeDependenciesToolset Condition=" '$(NativeDependenciesToolset)' == '' ">v120</NativeDependenciesToolset>
<NativeDependenciesPlatform Condition=" '$(NativeDependenciesPlatform)' == '' ">Win32</NativeDependenciesPlatform>
<NativeDependenciesConfiguration Condition=" '$(NativeDependenciesConfiguration)' == '' ">Debug</NativeDependenciesConfiguration>
<ZlibCallingConvention Condition=" '$(ZlibCallingConvention)' == '' ">cdecl</ZlibCallingConvention>
</PropertyGroup>
</Project>

@ -0,0 +1,14 @@
<?xml version="1.0" encoding="utf-8"?>
<Project ToolsVersion="4.0" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
<Choose>
<!-- Under older versions of Monodevelop, Choose is not supported and is just ignored, which gives us the desired effect. -->
<When Condition=" '$(OS)' != 'Unix' ">
<ItemGroup Condition=" '$(CopyNativeDependencies)' == 'true' ">
<Content Include="$(MSBuildThisFileDirectory)..\..\build\native\bin\$(NativeDependenciesToolset)\$(NativeDependenciesPlatform)\$(NativeDependenciesConfiguration)\dynamic\$(ZlibCallingConvention)\zlib.dll">
<CopyToOutputDirectory>PreserveNewest</CopyToOutputDirectory>
</Content>
</ItemGroup>
</When>
<Otherwise />
</Choose>
</Project>

@ -0,0 +1 @@
#define { package-version : 1.2.8.9; }

@ -0,0 +1,26 @@

Microsoft Visual Studio Solution File, Format Version 12.00
# Visual Studio 2012
Project("{8BC9CEB8-8B4A-11D0-8D11-00A0C91BC942}") = "zlib", "zlib.vcxproj", "{63BED288-E8C3-4345-B84D-2E64598DCF3A}"
EndProject
Global
GlobalSection(SolutionConfigurationPlatforms) = preSolution
Debug|Win32 = Debug|Win32
Release|Win32 = Release|Win32
Debug|x64 = Debug|x64
Release|x64 = Release|x64
EndGlobalSection
GlobalSection(ProjectConfigurationPlatforms) = postSolution
{63BED288-E8C3-4345-B84D-2E64598DCF3A}.Debug|Win32.ActiveCfg = Debug|Win32
{63BED288-E8C3-4345-B84D-2E64598DCF3A}.Debug|Win32.Build.0 = Debug|Win32
{63BED288-E8C3-4345-B84D-2E64598DCF3A}.Release|Win32.ActiveCfg = Release|Win32
{63BED288-E8C3-4345-B84D-2E64598DCF3A}.Release|Win32.Build.0 = Release|Win32
{63BED288-E8C3-4345-B84D-2E64598DCF3A}.Debug|x64.ActiveCfg = Debug|x64
{63BED288-E8C3-4345-B84D-2E64598DCF3A}.Debug|x64.Build.0 = Debug|x64
{63BED288-E8C3-4345-B84D-2E64598DCF3A}.Release|x64.ActiveCfg = Release|x64
{63BED288-E8C3-4345-B84D-2E64598DCF3A}.Release|x64.Build.0 = Release|x64
EndGlobalSection
GlobalSection(SolutionProperties) = preSolution
HideSolutionNode = FALSE
EndGlobalSection
EndGlobal

@ -0,0 +1,147 @@
<?xml version="1.0" encoding="utf-8"?>
<Project DefaultTargets="PreBuildCmds;PreBuildTargets;Build;PostBuildCmds;PostBuildTargets" ToolsVersion="4.0" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
<PropertyGroup Label="CoApp" >
<CoAppEtcDirectory>$(registry:HKEY_LOCAL_MACHINE\Software\Outercurve\CoApp.Powershell\etc)</CoAppEtcDirectory>
<!-- Set to true to make visual studio use PTK when building.-->
<UsePTKFromVisualStudio>false</UsePTKFromVisualStudio>
</PropertyGroup>
<PropertyGroup Label="Configuration" >
<!-- This lets Visual Studio see this as a VC12 project by default -->
<PlatformToolset Condition="'$(PlatformToolset)' == ''">v120</PlatformToolset>
</PropertyGroup>
<Import Condition="Exists('$(CoAppEtcDirectory)\common-variables.vcxproj')" Project="$(CoAppEtcDirectory)\common-variables.vcxproj" />
<PropertyGroup Label="CustomSettings" />
<PropertyGroup Label="Globals">
<ProjectGuid>{63BED288-E8C3-4345-B84D-2E64598DCF3A}</ProjectGuid>
<RootNamespace>$(MSBuildProjectName)</RootNamespace>
<OutNameSuffix Condition="$(IS_CDECL) AND $(IS_DYNAMIC)">1</OutNameSuffix>
<!-- set to Application or DynamicLibrary (DynamicLibrary can get altered to StaticLibrary by PTK -->
<ConfigurationType>DynamicLibrary</ConfigurationType>
<!-- Common Compiler Defines (semicolon delimited) -->
<Defines>_CRT_NONSTDC_NO_DEPRECATE;_CRT_SECURE_NO_DEPRECATE;_CRT_NONSTDC_NO_WARNINGS;</Defines>
<Defines Condition="$(IS_STDCALL)">$(Defines);ZLIB_WINAPI;</Defines>
<Defines Condition="$(IS_DYNAMIC)">$(Defines);ZLIB_DLL;</Defines>
<Defines Condition="'$(AppContainer)' == 'App'">$(Defines);IOWIN32_USING_WINRT_API=1;</Defines>
<Defines Condition="'$(AppContainer)' != 'App'">$(Defines);IOWIN32_USING_WINRT_API=0;</Defines>
<Defines Condition="'$(UseASM)' == 'ASM'">$(Defines);ASMV;ASMINF;</Defines>
<Defines Condition="$(IS_X64)">$(Defines);WIN64;</Defines>
<!-- Additional Include folders (semicolon delimited) -->
<IncludeDirectories>..\..\..\third_party\zlib;</IncludeDirectories>
<!-- Additional Library folders (semicolon delimited) -->
<LibraryDirectories></LibraryDirectories>
<!-- Libraries to Link with -->
<Libraries></Libraries>
<!-- Batch script to run before Build-->
<PreBuild></PreBuild>
<!-- Batch script to run after Build-->
<PostBuild></PostBuild>
<!-- Batch script to run before Link step-->
<PreLink></PreLink>
<!-- Batch script to run after Link-->
<PostLink></PostLink>
<!-- Batch script to run before Lib step-->
<PreLib></PreLib>
<!-- Batch script to run after Lib-->
<PostLib></PostLib>
<!-- Targets to run before Build (semcolon delimited)-->
<PreBuildTargets></PreBuildTargets>
<!-- Targets to run before Build (semcolon delimited)-->
<PostBuildTargets></PostBuildTargets>
<!-- for Dynamic libs, you can specify the Module .DEF file path -->
<ModuleDefinitionFile></ModuleDefinitionFile>
</PropertyGroup>
<Import Condition="Exists('$(CoAppEtcDirectory)\common-header.vcxproj')" Project="$(CoAppEtcDirectory)\common-header.vcxproj" />
<Import Condition="'$(UseASM)' == 'ASM'" Project="$(VCTargetsPath)\BuildCustomizations\masm.props" />
<PropertyGroup>
<OutDir>$(ProjectRootDir)Output/$(PlatformToolset)/$(Platform)/$(Configuration)/$(UsesConfigurationType)/$(CallingConvention)/$(CharacterSet)/$(AppContainer)/$(UseASM)/</OutDir>
<IntDir>$(ProjectRootDir)Intermediate/$(TargetName)/$(PlatformToolset)/$(Platform)/$(Configuration)/$(UsesConfigurationType)/$(CallingConvention)/$(CharacterSet)/$(AppContainer)/$(UseASM)/</IntDir>
</PropertyGroup>
<ItemDefinitionGroup>
<ClCompile>
<SDLCheck>true</SDLCheck>
</ClCompile>
<Link>
<BaseAddress>0x5A4C0000</BaseAddress>
</Link>
</ItemDefinitionGroup>
<ItemGroup Condition="'$(UseASM)' == 'ASM'">
<MASM Include="..\..\..\third_party\zlib\contrib\masmx64\gvmat64.asm" Condition="$(IS_X64)">
<AssembledCodeListingFile>$(IntDir)gvmat64.lst</AssembledCodeListingFile>
<MASMBeforeTargets>Build</MASMBeforeTargets>
<ObjectFileName>$(IntDir)gvmat64.obj</ObjectFileName>
</MASM>
<MASM Include="..\..\..\third_party\zlib\contrib\masmx64\inffasx64.asm" Condition="$(IS_X64)">
<AssembledCodeListingFile>$(IntDir)inffasx64.lst</AssembledCodeListingFile>
<MASMBeforeTargets>Build</MASMBeforeTargets>
<ObjectFileName>$(IntDir)inffasx64.obj</ObjectFileName>
</MASM>
<MASM Include="..\..\..\third_party\zlib\contrib\masmx86\inffas32.asm" Condition="$(IS_X86) AND $(IS_STDCALL)">
<AssembledCodeListingFile>$(IntDir)inffas32.lst</AssembledCodeListingFile>
<MASMBeforeTargets>Build</MASMBeforeTargets>
<UseSafeExceptionHandlers>true</UseSafeExceptionHandlers>
<AdditionalOptions>/coff %(AdditionalOptions)</AdditionalOptions>
<ObjectFileName>$(IntDir)inffas32.obj</ObjectFileName>
</MASM>
<MASM Include="..\..\..\third_party\zlib\contrib\masmx86\match686.asm" Condition="$(IS_X86) AND $(IS_STDCALL)">
<AssembledCodeListingFile>$(IntDir)match686.lst</AssembledCodeListingFile>
<MASMBeforeTargets>Build</MASMBeforeTargets>
<UseSafeExceptionHandlers>true</UseSafeExceptionHandlers>
<AdditionalOptions>/coff %(AdditionalOptions)</AdditionalOptions>
<ObjectFileName>$(IntDir)match686.obj</ObjectFileName>
</MASM>
</ItemGroup>
<ItemGroup Label="C Source Files">
<!-- Include the source files to compile here -->
<!-- <ClCompile Include="..\src\foo.c" /> -->
<ClCompile Include="..\..\..\third_party\zlib\adler32.c" />
<ClCompile Include="..\..\..\third_party\zlib\compress.c" />
<ClCompile Include="..\..\..\third_party\zlib\crc32.c" />
<ClCompile Include="..\..\..\third_party\zlib\deflate.c" />
<ClCompile Include="..\..\..\third_party\zlib\gzclose.c" />
<ClCompile Include="..\..\..\third_party\zlib\gzlib.c" />
<ClCompile Include="..\..\..\third_party\zlib\gzread.c" />
<ClCompile Include="..\..\..\third_party\zlib\gzwrite.c" />
<ClCompile Include="..\..\..\third_party\zlib\infback.c" />
<ClCompile Include="..\..\..\third_party\zlib\contrib\masmx64\inffas8664.c" Condition="$(IS_X64)" />
<ClCompile Include="..\..\..\third_party\zlib\inffast.c" />
<ClCompile Include="..\..\..\third_party\zlib\inflate.c" />
<ClCompile Include="..\..\..\third_party\zlib\inftrees.c" />
<ClCompile Include="..\..\..\third_party\zlib\trees.c" />
<ClCompile Include="..\..\..\third_party\zlib\uncompr.c" />
<ClCompile Include="..\..\..\third_party\zlib\zutil.c" />
</ItemGroup>
<ItemGroup Label="Resource Files">
<!-- Include the source files to compile here -->
<!-- <ResourceCompile Include="..\src\foo.rc" /> -->
<ResourceCompile Include="..\..\..\third_party\zlib\win32\zlib1.rc" />
</ItemGroup>
<Import Condition="'$(UseASM)' == 'ASM'" Project="$(VCTargetsPath)\BuildCustomizations\masm.targets" />
<Import Condition="Exists('$(CoAppEtcDirectory)\common-footer.vcxproj')" Project="$(CoAppEtcDirectory)\common-footer.vcxproj" />
</Project>
Loading…
Cancel
Save