You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ta...@apache.org on 2019/04/04 17:09:06 UTC

[impala] 04/12: IMPALA-7006: Add KRPC folders from kudu@334ecafd

This is an automated email from the ASF dual-hosted git repository.

tarmstrong pushed a commit to branch 2.x
in repository https://gitbox.apache.org/repos/asf/impala.git

commit dfb9e16960f858e1dccd209e7b1f7e4be60bc6d4
Author: Lars Volker <lv...@cloudera.com>
AuthorDate: Tue Jul 3 15:10:52 2018 -0700

    IMPALA-7006: Add KRPC folders from kudu@334ecafd
    
    cp -a ~/checkout/kudu/src/kudu/{rpc,util,security} be/src/kudu/
    
    Change-Id: I232db2b4ccf5df9aca87b21dea31bfb2735d1ab7
    Reviewed-on: http://gerrit.cloudera.org:8080/10757
    Reviewed-by: Lars Volker <lv...@cloudera.com>
    Tested-by: Lars Volker <lv...@cloudera.com>
---
 be/src/kudu/rpc/CMakeLists.txt                     |  138 ++
 be/src/kudu/rpc/acceptor_pool.cc                   |  175 ++
 be/src/kudu/rpc/acceptor_pool.h                    |   84 +
 be/src/kudu/rpc/blocking_ops.cc                    |  126 +
 be/src/kudu/rpc/blocking_ops.h                     |   58 +
 be/src/kudu/rpc/client_negotiation.cc              |  853 +++++++
 be/src/kudu/rpc/client_negotiation.h               |  263 +++
 be/src/kudu/rpc/connection.cc                      |  767 ++++++
 be/src/kudu/rpc/connection.h                       |  391 ++++
 be/src/kudu/rpc/connection_id.cc                   |   85 +
 be/src/kudu/rpc/connection_id.h                    |   84 +
 be/src/kudu/rpc/constants.cc                       |   37 +
 be/src/kudu/rpc/constants.h                        |   60 +
 be/src/kudu/rpc/exactly_once_rpc-test.cc           |  629 +++++
 be/src/kudu/rpc/inbound_call.cc                    |  345 +++
 be/src/kudu/rpc/inbound_call.h                     |  286 +++
 be/src/kudu/rpc/messenger.cc                       |  502 ++++
 be/src/kudu/rpc/messenger.h                        |  460 ++++
 be/src/kudu/rpc/mt-rpc-test.cc                     |  318 +++
 be/src/kudu/rpc/negotiation-test.cc                | 1346 +++++++++++
 be/src/kudu/rpc/negotiation.cc                     |  327 +++
 be/src/kudu/rpc/negotiation.h                      |   58 +
 be/src/kudu/rpc/outbound_call.cc                   |  531 +++++
 be/src/kudu/rpc/outbound_call.h                    |  348 +++
 be/src/kudu/rpc/periodic-test.cc                   |  295 +++
 be/src/kudu/rpc/periodic.cc                        |  219 ++
 be/src/kudu/rpc/periodic.h                         |  215 ++
 be/src/kudu/rpc/protoc-gen-krpc.cc                 |  691 ++++++
 be/src/kudu/rpc/proxy.cc                           |  116 +
 be/src/kudu/rpc/proxy.h                            |  126 +
 be/src/kudu/rpc/reactor-test.cc                    |  112 +
 be/src/kudu/rpc/reactor.cc                         |  918 ++++++++
 be/src/kudu/rpc/reactor.h                          |  427 ++++
 be/src/kudu/rpc/remote_method.cc                   |   53 +
 be/src/kudu/rpc/remote_method.h                    |   51 +
 be/src/kudu/rpc/remote_user.cc                     |   40 +
 be/src/kudu/rpc/remote_user.h                      |   99 +
 be/src/kudu/rpc/request_tracker-test.cc            |   86 +
 be/src/kudu/rpc/request_tracker.cc                 |   55 +
 be/src/kudu/rpc/request_tracker.h                  |   87 +
 be/src/kudu/rpc/response_callback.h                |   31 +
 be/src/kudu/rpc/result_tracker.cc                  |  595 +++++
 be/src/kudu/rpc/result_tracker.h                   |  401 ++++
 be/src/kudu/rpc/retriable_rpc.h                    |  296 +++
 be/src/kudu/rpc/rpc-bench.cc                       |  298 +++
 be/src/kudu/rpc/rpc-test-base.h                    |  661 ++++++
 be/src/kudu/rpc/rpc-test.cc                        | 1364 +++++++++++
 be/src/kudu/rpc/rpc.cc                             |  101 +
 be/src/kudu/rpc/rpc.h                              |  221 ++
 be/src/kudu/rpc/rpc_context.cc                     |  217 ++
 be/src/kudu/rpc/rpc_context.h                      |  245 ++
 be/src/kudu/rpc/rpc_controller.cc                  |  177 ++
 be/src/kudu/rpc/rpc_controller.h                   |  282 +++
 be/src/kudu/rpc/rpc_header.proto                   |  365 +++
 be/src/kudu/rpc/rpc_introspection.proto            |  110 +
 be/src/kudu/rpc/rpc_service.h                      |   47 +
 be/src/kudu/rpc/rpc_sidecar.cc                     |  115 +
 be/src/kudu/rpc/rpc_sidecar.h                      |   73 +
 be/src/kudu/rpc/rpc_stub-test.cc                   |  726 ++++++
 be/src/kudu/rpc/rpcz_store.cc                      |  272 +++
 be/src/kudu/rpc/rpcz_store.h                       |   74 +
 be/src/kudu/rpc/rtest.proto                        |  160 ++
 be/src/kudu/rpc/rtest_diff_package.proto           |   26 +
 be/src/kudu/rpc/sasl_common.cc                     |  470 ++++
 be/src/kudu/rpc/sasl_common.h                      |  158 ++
 be/src/kudu/rpc/sasl_helper.cc                     |  134 ++
 be/src/kudu/rpc/sasl_helper.h                      |  109 +
 be/src/kudu/rpc/serialization.cc                   |  223 ++
 be/src/kudu/rpc/serialization.h                    |   88 +
 be/src/kudu/rpc/server_negotiation.cc              |  989 ++++++++
 be/src/kudu/rpc/server_negotiation.h               |  259 +++
 be/src/kudu/rpc/service_if.cc                      |  160 ++
 be/src/kudu/rpc/service_if.h                       |  134 ++
 be/src/kudu/rpc/service_pool.cc                    |  234 ++
 be/src/kudu/rpc/service_pool.h                     |  117 +
 be/src/kudu/rpc/service_queue-test.cc              |  151 ++
 be/src/kudu/rpc/service_queue.cc                   |  145 ++
 be/src/kudu/rpc/service_queue.h                    |  225 ++
 be/src/kudu/rpc/transfer.cc                        |  283 +++
 be/src/kudu/rpc/transfer.h                         |  212 ++
 be/src/kudu/rpc/user_credentials.cc                |   64 +
 be/src/kudu/rpc/user_credentials.h                 |   53 +
 be/src/kudu/security/CMakeLists.txt                |  141 ++
 be/src/kudu/security/ca/cert_management-test.cc    |  294 +++
 be/src/kudu/security/ca/cert_management.cc         |  423 ++++
 be/src/kudu/security/ca/cert_management.h          |  226 ++
 be/src/kudu/security/cert-test.cc                  |  165 ++
 be/src/kudu/security/cert.cc                       |  301 +++
 be/src/kudu/security/cert.h                        |  119 +
 be/src/kudu/security/crypto-test.cc                |  257 +++
 be/src/kudu/security/crypto.cc                     |  276 +++
 be/src/kudu/security/crypto.h                      |  103 +
 be/src/kudu/security/init.cc                       |  465 ++++
 be/src/kudu/security/init.h                        |   84 +
 be/src/kudu/security/kerberos_util.cc              |   37 +
 be/src/kudu/security/kerberos_util.h               |   29 +
 be/src/kudu/security/krb5_realm_override.cc        |  105 +
 be/src/kudu/security/openssl_util.cc               |  322 +++
 be/src/kudu/security/openssl_util.h                |  217 ++
 be/src/kudu/security/openssl_util_bio.h            |  129 ++
 be/src/kudu/security/security-test-util.cc         |  103 +
 be/src/kudu/security/security-test-util.h          |   56 +
 be/src/kudu/security/security_flags.cc             |   42 +
 be/src/kudu/security/security_flags.h              |   36 +
 be/src/kudu/security/simple_acl.cc                 |   89 +
 be/src/kudu/security/simple_acl.h                  |   58 +
 be/src/kudu/security/test/mini_kdc-test.cc         |  144 ++
 be/src/kudu/security/test/mini_kdc.cc              |  315 +++
 be/src/kudu/security/test/mini_kdc.h               |  134 ++
 be/src/kudu/security/test/test_certs.cc            |  969 ++++++++
 be/src/kudu/security/test/test_certs.h             |   86 +
 be/src/kudu/security/test/test_pass.cc             |   40 +
 be/src/kudu/security/test/test_pass.h              |   37 +
 be/src/kudu/security/tls_context.cc                |  520 +++++
 be/src/kudu/security/tls_context.h                 |  202 ++
 be/src/kudu/security/tls_handshake-test.cc         |  390 ++++
 be/src/kudu/security/tls_handshake.cc              |  274 +++
 be/src/kudu/security/tls_handshake.h               |  171 ++
 be/src/kudu/security/tls_socket-test.cc            |  366 +++
 be/src/kudu/security/tls_socket.cc                 |  185 ++
 be/src/kudu/security/tls_socket.h                  |   60 +
 be/src/kudu/security/token-test.cc                 |  677 ++++++
 be/src/kudu/security/token.proto                   |   97 +
 be/src/kudu/security/token_signer.cc               |  299 +++
 be/src/kudu/security/token_signer.h                |  316 +++
 be/src/kudu/security/token_signing_key.cc          |  110 +
 be/src/kudu/security/token_signing_key.h           |  103 +
 be/src/kudu/security/token_verifier.cc             |  173 ++
 be/src/kudu/security/token_verifier.h              |  126 +
 be/src/kudu/security/x509_check_host.cc            |  439 ++++
 be/src/kudu/security/x509_check_host.h             |   50 +
 be/src/kudu/util/CMakeLists.txt                    |  482 ++++
 be/src/kudu/util/alignment.h                       |   28 +
 be/src/kudu/util/array_view.h                      |  133 ++
 be/src/kudu/util/async_logger.cc                   |  151 ++
 be/src/kudu/util/async_logger.h                    |  206 ++
 be/src/kudu/util/async_util-test.cc                |  129 ++
 be/src/kudu/util/async_util.h                      |   99 +
 be/src/kudu/util/atomic-test.cc                    |  135 ++
 be/src/kudu/util/atomic.cc                         |   56 +
 be/src/kudu/util/atomic.h                          |  322 +++
 be/src/kudu/util/auto_release_pool.h               |   99 +
 be/src/kudu/util/barrier.h                         |   68 +
 be/src/kudu/util/bit-stream-utils.h                |  150 ++
 be/src/kudu/util/bit-stream-utils.inline.h         |  211 ++
 be/src/kudu/util/bit-util-test.cc                  |   45 +
 be/src/kudu/util/bit-util.h                        |   57 +
 be/src/kudu/util/bitmap-test.cc                    |  230 ++
 be/src/kudu/util/bitmap.cc                         |  136 ++
 be/src/kudu/util/bitmap.h                          |  219 ++
 be/src/kudu/util/blocking_queue-test.cc            |  249 ++
 be/src/kudu/util/blocking_queue.h                  |  256 ++
 be/src/kudu/util/bloom_filter-test.cc              |   92 +
 be/src/kudu/util/bloom_filter.cc                   |   89 +
 be/src/kudu/util/bloom_filter.h                    |  254 ++
 be/src/kudu/util/boost_mutex_utils.h               |   45 +
 be/src/kudu/util/cache-bench.cc                    |  191 ++
 be/src/kudu/util/cache-test.cc                     |  246 ++
 be/src/kudu/util/cache.cc                          |  572 +++++
 be/src/kudu/util/cache.h                           |  216 ++
 be/src/kudu/util/cache_metrics.cc                  |   69 +
 be/src/kudu/util/cache_metrics.h                   |   42 +
 be/src/kudu/util/callback_bind-test.cc             |  119 +
 be/src/kudu/util/coding-inl.h                      |  120 +
 be/src/kudu/util/coding.cc                         |  142 ++
 be/src/kudu/util/coding.h                          |  113 +
 be/src/kudu/util/compression/compression-test.cc   |   90 +
 be/src/kudu/util/compression/compression.proto     |   29 +
 be/src/kudu/util/compression/compression_codec.cc  |  286 +++
 be/src/kudu/util/compression/compression_codec.h   |   78 +
 be/src/kudu/util/condition_variable.cc             |  142 ++
 be/src/kudu/util/condition_variable.h              |  118 +
 be/src/kudu/util/countdown_latch-test.cc           |   74 +
 be/src/kudu/util/countdown_latch.h                 |  137 ++
 be/src/kudu/util/cow_object.cc                     |   34 +
 be/src/kudu/util/cow_object.h                      |  437 ++++
 be/src/kudu/util/crc-test.cc                       |  112 +
 be/src/kudu/util/crc.cc                            |   56 +
 be/src/kudu/util/crc.h                             |   43 +
 be/src/kudu/util/curl_util.cc                      |  130 ++
 be/src/kudu/util/curl_util.h                       |   92 +
 be/src/kudu/util/debug-util-test.cc                |  458 ++++
 be/src/kudu/util/debug-util.cc                     |  800 +++++++
 be/src/kudu/util/debug-util.h                      |  321 +++
 be/src/kudu/util/debug/leak_annotations.h          |   84 +
 be/src/kudu/util/debug/leakcheck_disabler.h        |   48 +
 be/src/kudu/util/debug/sanitizer_scopes.h          |   47 +
 be/src/kudu/util/debug/trace_event.h               | 1501 ++++++++++++
 be/src/kudu/util/debug/trace_event_impl.cc         | 2436 ++++++++++++++++++++
 be/src/kudu/util/debug/trace_event_impl.h          |  726 ++++++
 .../kudu/util/debug/trace_event_impl_constants.cc  |   14 +
 be/src/kudu/util/debug/trace_event_memory.h        |   28 +
 .../kudu/util/debug/trace_event_synthetic_delay.cc |  238 ++
 .../kudu/util/debug/trace_event_synthetic_delay.h  |  166 ++
 be/src/kudu/util/debug/trace_logging.h             |  132 ++
 be/src/kudu/util/debug/unwind_safeness.cc          |  164 ++
 be/src/kudu/util/debug/unwind_safeness.h           |   29 +
 be/src/kudu/util/debug_ref_counted.h               |   56 +
 be/src/kudu/util/decimal_util-test.cc              |   81 +
 be/src/kudu/util/decimal_util.cc                   |   89 +
 be/src/kudu/util/decimal_util.h                    |   69 +
 be/src/kudu/util/easy_json-test.cc                 |  106 +
 be/src/kudu/util/easy_json.cc                      |  212 ++
 be/src/kudu/util/easy_json.h                       |  190 ++
 be/src/kudu/util/env-test.cc                       | 1173 ++++++++++
 be/src/kudu/util/env.cc                            |   93 +
 be/src/kudu/util/env.h                             |  681 ++++++
 be/src/kudu/util/env_posix.cc                      | 1852 +++++++++++++++
 be/src/kudu/util/env_util-test.cc                  |  192 ++
 be/src/kudu/util/env_util.cc                       |  320 +++
 be/src/kudu/util/env_util.h                        |  112 +
 be/src/kudu/util/errno-test.cc                     |   50 +
 be/src/kudu/util/errno.cc                          |   52 +
 be/src/kudu/util/errno.h                           |   36 +
 be/src/kudu/util/faststring-test.cc                |   65 +
 be/src/kudu/util/faststring.cc                     |   72 +
 be/src/kudu/util/faststring.h                      |  259 +++
 be/src/kudu/util/fault_injection.cc                |   78 +
 be/src/kudu/util/fault_injection.h                 |   98 +
 be/src/kudu/util/file_cache-stress-test.cc         |  402 ++++
 be/src/kudu/util/file_cache-test-util.h            |   92 +
 be/src/kudu/util/file_cache-test.cc                |  361 +++
 be/src/kudu/util/file_cache.cc                     |  654 ++++++
 be/src/kudu/util/file_cache.h                      |  209 ++
 be/src/kudu/util/flag_tags-test.cc                 |  135 ++
 be/src/kudu/util/flag_tags.cc                      |   91 +
 be/src/kudu/util/flag_tags.h                       |  169 ++
 be/src/kudu/util/flag_validators-test.cc           |  252 ++
 be/src/kudu/util/flag_validators.cc                |   67 +
 be/src/kudu/util/flag_validators.h                 |  102 +
 be/src/kudu/util/flags-test.cc                     |  109 +
 be/src/kudu/util/flags.cc                          |  604 +++++
 be/src/kudu/util/flags.h                           |   89 +
 be/src/kudu/util/group_varint-inl.h                |  294 +++
 be/src/kudu/util/group_varint-test.cc              |  144 ++
 be/src/kudu/util/group_varint.cc                   |   81 +
 be/src/kudu/util/hash_util-test.cc                 |   42 +
 be/src/kudu/util/hash_util.h                       |   71 +
 be/src/kudu/util/hdr_histogram-test.cc             |  116 +
 be/src/kudu/util/hdr_histogram.cc                  |  501 ++++
 be/src/kudu/util/hdr_histogram.h                   |  351 +++
 be/src/kudu/util/hexdump.cc                        |   85 +
 be/src/kudu/util/hexdump.h                         |   34 +
 be/src/kudu/util/high_water_mark.h                 |   85 +
 be/src/kudu/util/histogram.proto                   |   48 +
 be/src/kudu/util/init.cc                           |   89 +
 be/src/kudu/util/init.h                            |   33 +
 be/src/kudu/util/inline_slice-test.cc              |   88 +
 be/src/kudu/util/inline_slice.h                    |  181 ++
 be/src/kudu/util/int128-test.cc                    |   69 +
 be/src/kudu/util/int128.h                          |   46 +
 be/src/kudu/util/int128_util.h                     |   39 +
 be/src/kudu/util/interval_tree-inl.h               |  444 ++++
 be/src/kudu/util/interval_tree-test.cc             |  353 +++
 be/src/kudu/util/interval_tree.h                   |  158 ++
 be/src/kudu/util/jsonreader-test.cc                |  193 ++
 be/src/kudu/util/jsonreader.cc                     |  141 ++
 be/src/kudu/util/jsonreader.h                      |   92 +
 be/src/kudu/util/jsonwriter-test.cc                |  216 ++
 be/src/kudu/util/jsonwriter.cc                     |  352 +++
 be/src/kudu/util/jsonwriter.h                      |  102 +
 be/src/kudu/util/jsonwriter_test.proto             |   79 +
 be/src/kudu/util/kernel_stack_watchdog.cc          |  256 ++
 be/src/kudu/util/kernel_stack_watchdog.h           |  290 +++
 be/src/kudu/util/knapsack_solver-test.cc           |  172 ++
 be/src/kudu/util/knapsack_solver.h                 |  269 +++
 be/src/kudu/util/locks.cc                          |   47 +
 be/src/kudu/util/locks.h                           |  294 +++
 be/src/kudu/util/logging-test.cc                   |  249 ++
 be/src/kudu/util/logging.cc                        |  413 ++++
 be/src/kudu/util/logging.h                         |  367 +++
 be/src/kudu/util/logging_callback.h                |   46 +
 be/src/kudu/util/logging_test_util.h               |   60 +
 be/src/kudu/util/maintenance_manager-test.cc       |  369 +++
 be/src/kudu/util/maintenance_manager.cc            |  550 +++++
 be/src/kudu/util/maintenance_manager.h             |  361 +++
 be/src/kudu/util/maintenance_manager.proto         |   54 +
 be/src/kudu/util/make_shared.h                     |   64 +
 be/src/kudu/util/malloc.cc                         |   35 +
 be/src/kudu/util/malloc.h                          |   32 +
 be/src/kudu/util/map-util-test.cc                  |  116 +
 be/src/kudu/util/mem_tracker-test.cc               |  285 +++
 be/src/kudu/util/mem_tracker.cc                    |  296 +++
 be/src/kudu/util/mem_tracker.h                     |  272 +++
 be/src/kudu/util/memcmpable_varint-test.cc         |  220 ++
 be/src/kudu/util/memcmpable_varint.cc              |  257 +++
 be/src/kudu/util/memcmpable_varint.h               |   45 +
 be/src/kudu/util/memory/arena-test.cc              |  205 ++
 be/src/kudu/util/memory/arena.cc                   |  167 ++
 be/src/kudu/util/memory/arena.h                    |  501 ++++
 be/src/kudu/util/memory/memory.cc                  |  339 +++
 be/src/kudu/util/memory/memory.h                   |  970 ++++++++
 be/src/kudu/util/memory/overwrite.cc               |   42 +
 be/src/kudu/util/memory/overwrite.h                |   33 +
 be/src/kudu/util/metrics-test.cc                   |  388 ++++
 be/src/kudu/util/metrics.cc                        |  746 ++++++
 be/src/kudu/util/metrics.h                         | 1195 ++++++++++
 be/src/kudu/util/minidump-test.cc                  |  149 ++
 be/src/kudu/util/minidump.cc                       |  382 +++
 be/src/kudu/util/minidump.h                        |  104 +
 be/src/kudu/util/monotime-test.cc                  |  424 ++++
 be/src/kudu/util/monotime.cc                       |  334 +++
 be/src/kudu/util/monotime.h                        |  421 ++++
 be/src/kudu/util/mt-hdr_histogram-test.cc          |  116 +
 be/src/kudu/util/mt-metrics-test.cc                |  128 +
 be/src/kudu/util/mt-threadlocal-test.cc            |  357 +++
 be/src/kudu/util/mutex.cc                          |  164 ++
 be/src/kudu/util/mutex.h                           |  142 ++
 be/src/kudu/util/net/dns_resolver-test.cc          |   59 +
 be/src/kudu/util/net/dns_resolver.cc               |   65 +
 be/src/kudu/util/net/dns_resolver.h                |   62 +
 be/src/kudu/util/net/net_util-test.cc              |  170 ++
 be/src/kudu/util/net/net_util.cc                   |  402 ++++
 be/src/kudu/util/net/net_util.h                    |  166 ++
 be/src/kudu/util/net/sockaddr.cc                   |  136 ++
 be/src/kudu/util/net/sockaddr.h                    |   94 +
 be/src/kudu/util/net/socket-test.cc                |   89 +
 be/src/kudu/util/net/socket.cc                     |  590 +++++
 be/src/kudu/util/net/socket.h                      |  178 ++
 be/src/kudu/util/nvm_cache.cc                      |  577 +++++
 be/src/kudu/util/nvm_cache.h                       |   31 +
 be/src/kudu/util/object_pool-test.cc               |   86 +
 be/src/kudu/util/object_pool.h                     |  166 ++
 be/src/kudu/util/oid_generator-test.cc             |   52 +
 be/src/kudu/util/oid_generator.cc                  |   65 +
 be/src/kudu/util/oid_generator.h                   |   63 +
 be/src/kudu/util/once-test.cc                      |  113 +
 be/src/kudu/util/once.cc                           |   32 +
 be/src/kudu/util/once.h                            |  116 +
 be/src/kudu/util/os-util-test.cc                   |   62 +
 be/src/kudu/util/os-util.cc                        |  185 ++
 be/src/kudu/util/os-util.h                         |   72 +
 be/src/kudu/util/path_util-test.cc                 |   77 +
 be/src/kudu/util/path_util.cc                      |  122 +
 be/src/kudu/util/path_util.h                       |   63 +
 be/src/kudu/util/pb_util-internal.cc               |  105 +
 be/src/kudu/util/pb_util-internal.h                |  136 ++
 be/src/kudu/util/pb_util-test.cc                   |  661 ++++++
 be/src/kudu/util/pb_util.cc                        | 1088 +++++++++
 be/src/kudu/util/pb_util.h                         |  513 +++++
 be/src/kudu/util/pb_util.proto                     |   45 +
 be/src/kudu/util/pb_util_test.proto                |   29 +
 be/src/kudu/util/process_memory-test.cc            |   75 +
 be/src/kudu/util/process_memory.cc                 |  287 +++
 be/src/kudu/util/process_memory.h                  |   62 +
 be/src/kudu/util/promise.h                         |   79 +
 be/src/kudu/util/proto_container_test.proto        |   25 +
 be/src/kudu/util/proto_container_test2.proto       |   29 +
 be/src/kudu/util/proto_container_test3.proto       |   33 +
 be/src/kudu/util/protobuf-annotations.h            |   33 +
 be/src/kudu/util/protobuf_util.h                   |   39 +
 be/src/kudu/util/protoc-gen-insertions.cc          |   77 +
 be/src/kudu/util/pstack_watcher-test.cc            |  100 +
 be/src/kudu/util/pstack_watcher.cc                 |  249 ++
 be/src/kudu/util/pstack_watcher.h                  |  101 +
 be/src/kudu/util/random-test.cc                    |  171 ++
 be/src/kudu/util/random.h                          |  252 ++
 be/src/kudu/util/random_util-test.cc               |   75 +
 be/src/kudu/util/random_util.cc                    |   65 +
 be/src/kudu/util/random_util.h                     |   44 +
 be/src/kudu/util/rle-encoding.h                    |  523 +++++
 be/src/kudu/util/rle-test.cc                       |  546 +++++
 be/src/kudu/util/rolling_log-test.cc               |  147 ++
 be/src/kudu/util/rolling_log.cc                    |  285 +++
 be/src/kudu/util/rolling_log.h                     |  128 +
 be/src/kudu/util/rw_mutex-test.cc                  |  185 ++
 be/src/kudu/util/rw_mutex.cc                       |  207 ++
 be/src/kudu/util/rw_mutex.h                        |  123 +
 be/src/kudu/util/rw_semaphore-test.cc              |   94 +
 be/src/kudu/util/rw_semaphore.h                    |  206 ++
 be/src/kudu/util/rwc_lock-test.cc                  |  147 ++
 be/src/kudu/util/rwc_lock.cc                       |  136 ++
 be/src/kudu/util/rwc_lock.h                        |  142 ++
 be/src/kudu/util/safe_math-test.cc                 |   56 +
 be/src/kudu/util/safe_math.h                       |   69 +
 be/src/kudu/util/scoped_cleanup-test.cc            |   56 +
 be/src/kudu/util/scoped_cleanup.h                  |   67 +
 be/src/kudu/util/semaphore.cc                      |  105 +
 be/src/kudu/util/semaphore.h                       |   77 +
 be/src/kudu/util/semaphore_macosx.cc               |   75 +
 be/src/kudu/util/signal.cc                         |   47 +
 be/src/kudu/util/signal.h                          |   42 +
 be/src/kudu/util/slice-test.cc                     |   61 +
 be/src/kudu/util/slice.cc                          |   97 +
 be/src/kudu/util/slice.h                           |  332 +++
 .../util/sorted_disjoint_interval_list-test.cc     |   98 +
 be/src/kudu/util/sorted_disjoint_interval_list.h   |   95 +
 be/src/kudu/util/spinlock_profiling-test.cc        |   81 +
 be/src/kudu/util/spinlock_profiling.cc             |  308 +++
 be/src/kudu/util/spinlock_profiling.h              |   72 +
 be/src/kudu/util/stack_watchdog-test.cc            |  152 ++
 be/src/kudu/util/status-test.cc                    |  119 +
 be/src/kudu/util/status.cc                         |  170 ++
 be/src/kudu/util/status.h                          |  493 ++++
 be/src/kudu/util/status_callback.cc                |   41 +
 be/src/kudu/util/status_callback.h                 |   54 +
 be/src/kudu/util/stopwatch.h                       |  364 +++
 be/src/kudu/util/string_case-test.cc               |   65 +
 be/src/kudu/util/string_case.cc                    |   76 +
 be/src/kudu/util/string_case.h                     |   48 +
 be/src/kudu/util/striped64-test.cc                 |  163 ++
 be/src/kudu/util/striped64.cc                      |  191 ++
 be/src/kudu/util/striped64.h                       |  168 ++
 be/src/kudu/util/subprocess-test.cc                |  381 +++
 be/src/kudu/util/subprocess.cc                     |  815 +++++++
 be/src/kudu/util/subprocess.h                      |  219 ++
 be/src/kudu/util/test_graph.cc                     |  121 +
 be/src/kudu/util/test_graph.h                      |   90 +
 be/src/kudu/util/test_macros.h                     |  123 +
 be/src/kudu/util/test_main.cc                      |  109 +
 be/src/kudu/util/test_util.cc                      |  446 ++++
 be/src/kudu/util/test_util.h                       |  146 ++
 be/src/kudu/util/test_util_prod.cc                 |   28 +
 be/src/kudu/util/test_util_prod.h                  |   32 +
 be/src/kudu/util/thread-test.cc                    |  160 ++
 be/src/kudu/util/thread.cc                         |  628 +++++
 be/src/kudu/util/thread.h                          |  373 +++
 be/src/kudu/util/thread_restrictions.cc            |   87 +
 be/src/kudu/util/thread_restrictions.h             |  121 +
 be/src/kudu/util/threadlocal.cc                    |   89 +
 be/src/kudu/util/threadlocal.h                     |  128 +
 be/src/kudu/util/threadlocal_cache.h               |  110 +
 be/src/kudu/util/threadpool-test.cc                |  941 ++++++++
 be/src/kudu/util/threadpool.cc                     |  766 ++++++
 be/src/kudu/util/threadpool.h                      |  505 ++++
 be/src/kudu/util/throttler-test.cc                 |   76 +
 be/src/kudu/util/throttler.cc                      |   67 +
 be/src/kudu/util/throttler.h                       |   62 +
 be/src/kudu/util/trace-test.cc                     |  891 +++++++
 be/src/kudu/util/trace.cc                          |  259 +++
 be/src/kudu/util/trace.h                           |  292 +++
 be/src/kudu/util/trace_metrics.cc                  |   74 +
 be/src/kudu/util/trace_metrics.h                   |   89 +
 be/src/kudu/util/url-coding-test.cc                |  112 +
 be/src/kudu/util/url-coding.cc                     |  208 ++
 be/src/kudu/util/url-coding.h                      |   69 +
 be/src/kudu/util/user-test.cc                      |   44 +
 be/src/kudu/util/user.cc                           |   90 +
 be/src/kudu/util/user.h                            |   32 +
 be/src/kudu/util/version_info.cc                   |   84 +
 be/src/kudu/util/version_info.h                    |   51 +
 be/src/kudu/util/version_info.proto                |   32 +
 be/src/kudu/util/version_util-test.cc              |   66 +
 be/src/kudu/util/version_util.cc                   |   83 +
 be/src/kudu/util/version_util.h                    |   58 +
 be/src/kudu/util/web_callback_registry.h           |  129 ++
 be/src/kudu/util/website_util.cc                   |   43 +
 be/src/kudu/util/website_util.h                    |   35 +
 be/src/kudu/util/zlib.cc                           |  127 +
 be/src/kudu/util/zlib.h                            |   39 +
 450 files changed, 99139 insertions(+)

diff --git a/be/src/kudu/rpc/CMakeLists.txt b/be/src/kudu/rpc/CMakeLists.txt
new file mode 100644
index 0000000..f8cdb02
--- /dev/null
+++ b/be/src/kudu/rpc/CMakeLists.txt
@@ -0,0 +1,138 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you 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.
+
+#### Global header protobufs
+PROTOBUF_GENERATE_CPP(
+  RPC_HEADER_PROTO_SRCS RPC_HEADER_PROTO_HDRS RPC_HEADER_PROTO_TGTS
+  SOURCE_ROOT ${CMAKE_CURRENT_SOURCE_DIR}/../..
+  BINARY_ROOT ${CMAKE_CURRENT_BINARY_DIR}/../..
+  PROTO_FILES rpc_header.proto)
+ADD_EXPORTABLE_LIBRARY(rpc_header_proto
+  SRCS ${RPC_HEADER_PROTO_SRCS}
+  DEPS protobuf pb_util_proto token_proto
+  NONLINK_DEPS ${RPC_HEADER_PROTO_TGTS})
+
+PROTOBUF_GENERATE_CPP(
+  RPC_INTROSPECTION_PROTO_SRCS RPC_INTROSPECTION_PROTO_HDRS RPC_INTROSPECTION_PROTO_TGTS
+  SOURCE_ROOT ${CMAKE_CURRENT_SOURCE_DIR}/../..
+  BINARY_ROOT ${CMAKE_CURRENT_BINARY_DIR}/../..
+  PROTO_FILES rpc_introspection.proto)
+set(RPC_INTROSPECTION_PROTO_LIBS
+  rpc_header_proto
+  protobuf)
+ADD_EXPORTABLE_LIBRARY(rpc_introspection_proto
+  SRCS ${RPC_INTROSPECTION_PROTO_SRCS}
+  DEPS ${RPC_INTROSPECTION_PROTO_LIBS}
+  NONLINK_DEPS ${RPC_INTROSPECTION_PROTO_TGTS})
+
+### RPC library
+set(KRPC_SRCS
+    acceptor_pool.cc
+    blocking_ops.cc
+    client_negotiation.cc
+    connection.cc
+    connection_id.cc
+    constants.cc
+    inbound_call.cc
+    messenger.cc
+    negotiation.cc
+    outbound_call.cc
+    periodic.cc
+    proxy.cc
+    reactor.cc
+    remote_method.cc
+    remote_user.cc
+    request_tracker.cc
+    result_tracker.cc
+    rpc.cc
+    rpc_context.cc
+    rpc_controller.cc
+    rpc_sidecar.cc
+    rpcz_store.cc
+    sasl_common.cc
+    sasl_helper.cc
+    serialization.cc
+    server_negotiation.cc
+    service_if.cc
+    service_pool.cc
+    service_queue.cc
+    user_credentials.cc
+    transfer.cc
+)
+
+set(KRPC_LIBS
+  cyrus_sasl
+  gssapi_krb5
+  gutil
+  kudu_util
+  libev
+  rpc_header_proto
+  rpc_introspection_proto
+  security)
+
+ADD_EXPORTABLE_LIBRARY(krpc
+  SRCS ${KRPC_SRCS}
+  DEPS ${KRPC_LIBS})
+
+### RPC generator tool
+add_executable(protoc-gen-krpc protoc-gen-krpc.cc)
+target_link_libraries(protoc-gen-krpc
+    ${KUDU_BASE_LIBS}
+    rpc_header_proto
+    protoc
+    protobuf
+    gutil
+    kudu_util)
+
+#### RPC test
+PROTOBUF_GENERATE_CPP(
+  RPC_TEST_DIFF_PACKAGE_SRCS RPC_TEST_DIFF_PACKAGE_HDRS RPC_TEST_DIFF_PACKAGE_TGTS
+  SOURCE_ROOT ${CMAKE_CURRENT_SOURCE_DIR}/../..
+  BINARY_ROOT ${CMAKE_CURRENT_BINARY_DIR}/../..
+  PROTO_FILES rtest_diff_package.proto)
+add_library(rtest_diff_package_proto ${RPC_TEST_DIFF_PACKAGE_SRCS} ${RPC_TEST_DIFF_PACKAGE_HDRS})
+target_link_libraries(rtest_diff_package_proto rpc_header_proto)
+
+KRPC_GENERATE(
+  RTEST_KRPC_SRCS RTEST_KRPC_HDRS RTEST_KRPC_TGTS
+  SOURCE_ROOT ${CMAKE_CURRENT_SOURCE_DIR}/../..
+  BINARY_ROOT ${CMAKE_CURRENT_BINARY_DIR}/../..
+  PROTO_FILES rtest.proto)
+add_library(rtest_krpc ${RTEST_KRPC_SRCS} ${RTEST_KRPC_HDRS})
+target_link_libraries(rtest_krpc
+  krpc
+  rpc_header_proto
+  rtest_diff_package_proto)
+
+# Tests
+set(KUDU_TEST_LINK_LIBS
+  krpc
+  mini_kdc
+  rpc_header_proto
+  rtest_krpc
+  security_test_util
+  ${KUDU_MIN_TEST_LIBS})
+ADD_KUDU_TEST(exactly_once_rpc-test PROCESSORS 10)
+ADD_KUDU_TEST(mt-rpc-test RUN_SERIAL true)
+ADD_KUDU_TEST(negotiation-test)
+ADD_KUDU_TEST(periodic-test)
+ADD_KUDU_TEST(reactor-test)
+ADD_KUDU_TEST(request_tracker-test)
+ADD_KUDU_TEST(rpc-bench RUN_SERIAL true)
+ADD_KUDU_TEST(rpc-test)
+ADD_KUDU_TEST(rpc_stub-test)
+ADD_KUDU_TEST(service_queue-test RUN_SERIAL true)
diff --git a/be/src/kudu/rpc/acceptor_pool.cc b/be/src/kudu/rpc/acceptor_pool.cc
new file mode 100644
index 0000000..e4bcbd1
--- /dev/null
+++ b/be/src/kudu/rpc/acceptor_pool.cc
@@ -0,0 +1,175 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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.
+
+#include "kudu/rpc/acceptor_pool.h"
+
+#include <string>
+#include <ostream>
+#include <vector>
+
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+
+#include "kudu/gutil/basictypes.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/rpc/messenger.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/logging.h"
+#include "kudu/util/metrics.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/net/socket.h"
+#include "kudu/util/status.h"
+#include "kudu/util/thread.h"
+
+namespace google {
+namespace protobuf {
+
+class Message;
+
+}
+}
+
+using google::protobuf::Message;
+using std::string;
+
+METRIC_DEFINE_counter(server, rpc_connections_accepted,
+                      "RPC Connections Accepted",
+                      kudu::MetricUnit::kConnections,
+                      "Number of incoming TCP connections made to the RPC server");
+
+DEFINE_int32(rpc_acceptor_listen_backlog, 128,
+             "Socket backlog parameter used when listening for RPC connections. "
+             "This defines the maximum length to which the queue of pending "
+             "TCP connections inbound to the RPC server may grow. If a connection "
+             "request arrives when the queue is full, the client may receive "
+             "an error. Higher values may help the server ride over bursts of "
+             "new inbound connection requests.");
+TAG_FLAG(rpc_acceptor_listen_backlog, advanced);
+
+namespace kudu {
+namespace rpc {
+
+AcceptorPool::AcceptorPool(Messenger* messenger, Socket* socket,
+                           Sockaddr bind_address)
+    : messenger_(messenger),
+      socket_(socket->Release()),
+      bind_address_(bind_address),
+      rpc_connections_accepted_(METRIC_rpc_connections_accepted.Instantiate(
+          messenger->metric_entity())),
+      closing_(false) {}
+
+AcceptorPool::~AcceptorPool() {
+  Shutdown();
+}
+
+Status AcceptorPool::Start(int num_threads) {
+  RETURN_NOT_OK(socket_.Listen(FLAGS_rpc_acceptor_listen_backlog));
+
+  for (int i = 0; i < num_threads; i++) {
+    scoped_refptr<kudu::Thread> new_thread;
+    Status s = kudu::Thread::Create("acceptor pool", "acceptor",
+        &AcceptorPool::RunThread, this, &new_thread);
+    if (!s.ok()) {
+      Shutdown();
+      return s;
+    }
+    threads_.push_back(new_thread);
+  }
+  return Status::OK();
+}
+
+void AcceptorPool::Shutdown() {
+  if (Acquire_CompareAndSwap(&closing_, false, true) != false) {
+    VLOG(2) << "Acceptor Pool on " << bind_address_.ToString()
+            << " already shut down";
+    return;
+  }
+
+#if defined(__linux__)
+  // Closing the socket will break us out of accept() if we're in it, and
+  // prevent future accepts.
+  WARN_NOT_OK(socket_.Shutdown(true, true),
+              strings::Substitute("Could not shut down acceptor socket on $0",
+                                  bind_address_.ToString()));
+#else
+  // Calling shutdown on an accepting (non-connected) socket is illegal on most
+  // platforms (but not Linux). Instead, the accepting threads are interrupted
+  // forcefully.
+  for (const scoped_refptr<kudu::Thread>& thread : threads_) {
+    pthread_cancel(thread.get()->pthread_id());
+  }
+#endif
+
+  for (const scoped_refptr<kudu::Thread>& thread : threads_) {
+    CHECK_OK(ThreadJoiner(thread.get()).Join());
+  }
+  threads_.clear();
+
+  // Close the socket: keeping the descriptor open and, possibly, receiving late
+  // not-to-be-read messages from the peer does not make much sense. The
+  // Socket::Close() method is called upon destruction of the aggregated socket_
+  // object as well. However, the typical ownership pattern of an AcceptorPool
+  // object includes two references wrapped via a shared_ptr smart pointer: one
+  // is held by Messenger, another by RpcServer. If not calling Socket::Close()
+  // here, it would  necessary to wait until Messenger::Shutdown() is called for
+  // the corresponding messenger object to close this socket.
+  ignore_result(socket_.Close());
+}
+
+Sockaddr AcceptorPool::bind_address() const {
+  return bind_address_;
+}
+
+Status AcceptorPool::GetBoundAddress(Sockaddr* addr) const {
+  return socket_.GetSocketAddress(addr);
+}
+
+int64_t AcceptorPool::num_rpc_connections_accepted() const {
+  return rpc_connections_accepted_->value();
+}
+
+void AcceptorPool::RunThread() {
+  while (true) {
+    Socket new_sock;
+    Sockaddr remote;
+    VLOG(2) << "calling accept() on socket " << socket_.GetFd()
+            << " listening on " << bind_address_.ToString();
+    Status s = socket_.Accept(&new_sock, &remote, Socket::FLAG_NONBLOCKING);
+    if (!s.ok()) {
+      if (Release_Load(&closing_)) {
+        break;
+      }
+      KLOG_EVERY_N_SECS(WARNING, 1) << "AcceptorPool: accept failed: " << s.ToString()
+                                    << THROTTLE_MSG;
+      continue;
+    }
+    s = new_sock.SetNoDelay(true);
+    if (!s.ok()) {
+      KLOG_EVERY_N_SECS(WARNING, 1) << "Acceptor with remote = " << remote.ToString()
+          << " failed to set TCP_NODELAY on a newly accepted socket: "
+          << s.ToString() << THROTTLE_MSG;
+      continue;
+    }
+    rpc_connections_accepted_->Increment();
+    messenger_->RegisterInboundSocket(&new_sock, remote);
+  }
+  VLOG(1) << "AcceptorPool shutting down.";
+}
+
+} // namespace rpc
+} // namespace kudu
diff --git a/be/src/kudu/rpc/acceptor_pool.h b/be/src/kudu/rpc/acceptor_pool.h
new file mode 100644
index 0000000..ba1996a
--- /dev/null
+++ b/be/src/kudu/rpc/acceptor_pool.h
@@ -0,0 +1,84 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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.
+
+#ifndef KUDU_RPC_ACCEPTOR_POOL_H
+#define KUDU_RPC_ACCEPTOR_POOL_H
+
+#include <stdint.h>
+#include <vector>
+
+#include "kudu/gutil/atomicops.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/net/socket.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+class Counter;
+class Thread;
+
+namespace rpc {
+
+class Messenger;
+
+// A pool of threads calling accept() to create new connections.
+// Acceptor pool threads terminate when they notice that the messenger has been
+// shut down, if Shutdown() is called, or if the pool object is destructed.
+class AcceptorPool {
+ public:
+  // Create a new acceptor pool.  Calls socket::Release to take ownership of the
+  // socket.
+  // 'socket' must be already bound, but should not yet be listening.
+  AcceptorPool(Messenger *messenger, Socket *socket, Sockaddr bind_address);
+  ~AcceptorPool();
+
+  // Start listening and accepting connections.
+  Status Start(int num_threads);
+  void Shutdown();
+
+  // Return the address that the pool is bound to. If the port is specified as
+  // 0, then this will always return port 0.
+  Sockaddr bind_address() const;
+
+  // Return the address that the pool is bound to. This only works while the
+  // socket is open, and if the specified port is 0 then this will return the
+  // actual port that was bound.
+  Status GetBoundAddress(Sockaddr* addr) const;
+
+  // Return the number of connections accepted by this messenger. Thread-safe.
+  int64_t num_rpc_connections_accepted() const;
+
+ private:
+  void RunThread();
+
+  Messenger *messenger_;
+  Socket socket_;
+  Sockaddr bind_address_;
+  std::vector<scoped_refptr<kudu::Thread> > threads_;
+
+  scoped_refptr<Counter> rpc_connections_accepted_;
+
+  Atomic32 closing_;
+
+  DISALLOW_COPY_AND_ASSIGN(AcceptorPool);
+};
+
+} // namespace rpc
+} // namespace kudu
+#endif
diff --git a/be/src/kudu/rpc/blocking_ops.cc b/be/src/kudu/rpc/blocking_ops.cc
new file mode 100644
index 0000000..f5cd644
--- /dev/null
+++ b/be/src/kudu/rpc/blocking_ops.cc
@@ -0,0 +1,126 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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.
+
+#include "kudu/rpc/blocking_ops.h"
+
+#include <cstdint>
+#include <cstring>
+#include <ostream>
+
+#include <glog/logging.h>
+#include <google/protobuf/message_lite.h>
+
+#include "kudu/gutil/endian.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/rpc/constants.h"
+#include "kudu/rpc/serialization.h"
+#include "kudu/rpc/transfer.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/net/socket.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+namespace rpc {
+
+using google::protobuf::MessageLite;
+
+const char kHTTPHeader[] = "HTTP";
+
+Status CheckInBlockingMode(const Socket* sock) {
+  bool is_nonblocking;
+  RETURN_NOT_OK(sock->IsNonBlocking(&is_nonblocking));
+  if (is_nonblocking) {
+    static const char* const kErrMsg = "socket is not in blocking mode";
+    LOG(DFATAL) << kErrMsg;
+    return Status::IllegalState(kErrMsg);
+  }
+  return Status::OK();
+}
+
+Status SendFramedMessageBlocking(Socket* sock, const MessageLite& header, const MessageLite& msg,
+    const MonoTime& deadline) {
+  DCHECK(sock != nullptr);
+  DCHECK(header.IsInitialized()) << "header protobuf must be initialized";
+  DCHECK(msg.IsInitialized()) << "msg protobuf must be initialized";
+
+  // Ensure we are in blocking mode.
+  // These blocking calls are typically not in the fast path, so doing this for all build types.
+  RETURN_NOT_OK(CheckInBlockingMode(sock));
+
+  // Serialize message
+  faststring param_buf;
+  serialization::SerializeMessage(msg, &param_buf);
+
+  // Serialize header and initial length
+  faststring header_buf;
+  serialization::SerializeHeader(header, param_buf.size(), &header_buf);
+
+  // Write header & param to stream
+  size_t nsent;
+  RETURN_NOT_OK(sock->BlockingWrite(header_buf.data(), header_buf.size(), &nsent, deadline));
+  RETURN_NOT_OK(sock->BlockingWrite(param_buf.data(), param_buf.size(), &nsent, deadline));
+
+  return Status::OK();
+}
+
+Status ReceiveFramedMessageBlocking(Socket* sock, faststring* recv_buf,
+    MessageLite* header, Slice* param_buf, const MonoTime& deadline) {
+  DCHECK(sock != nullptr);
+  DCHECK(recv_buf != nullptr);
+  DCHECK(header != nullptr);
+  DCHECK(param_buf != nullptr);
+
+  RETURN_NOT_OK(CheckInBlockingMode(sock));
+
+  // Read the message prefix, which specifies the length of the payload.
+  recv_buf->clear();
+  recv_buf->resize(kMsgLengthPrefixLength);
+  size_t recvd = 0;
+  RETURN_NOT_OK(sock->BlockingRecv(recv_buf->data(), kMsgLengthPrefixLength, &recvd, deadline));
+  uint32_t payload_len = NetworkByteOrder::Load32(recv_buf->data());
+
+  // Verify that the payload size isn't out of bounds.
+  // This can happen because of network corruption, or a naughty client.
+  if (PREDICT_FALSE(payload_len > FLAGS_rpc_max_message_size)) {
+    // A common user mistake is to try to speak the Kudu RPC protocol to an
+    // HTTP endpoint, or vice versa.
+    if (memcmp(recv_buf->data(), kHTTPHeader, strlen(kHTTPHeader)) == 0) {
+      return Status::IOError(
+          "received invalid RPC message which appears to be an HTTP response. "
+          "Verify that you have specified a valid RPC port and not an HTTP port.");
+    }
+
+    return Status::IOError(
+        strings::Substitute(
+            "received invalid message of size $0 which exceeds"
+            " the rpc_max_message_size of $1 bytes",
+            payload_len, FLAGS_rpc_max_message_size));
+  }
+
+  // Read the message payload.
+  recvd = 0;
+  recv_buf->resize(payload_len + kMsgLengthPrefixLength);
+  RETURN_NOT_OK(sock->BlockingRecv(recv_buf->data() + kMsgLengthPrefixLength,
+                payload_len, &recvd, deadline));
+  RETURN_NOT_OK(serialization::ParseMessage(Slice(*recv_buf), header, param_buf));
+  return Status::OK();
+}
+
+} // namespace rpc
+} // namespace kudu
diff --git a/be/src/kudu/rpc/blocking_ops.h b/be/src/kudu/rpc/blocking_ops.h
new file mode 100644
index 0000000..b305ba7
--- /dev/null
+++ b/be/src/kudu/rpc/blocking_ops.h
@@ -0,0 +1,58 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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.
+
+#ifndef KUDU_RPC_BLOCKING_OPS_H
+#define KUDU_RPC_BLOCKING_OPS_H
+
+namespace google {
+namespace protobuf {
+class MessageLite;
+} // namespace protobuf
+} // namespace google
+
+namespace kudu {
+
+class faststring;
+class MonoTime;
+class Slice;
+class Socket;
+class Status;
+
+namespace rpc {
+
+// Returns OK if socket is in blocking mode. Otherwise, returns an error.
+Status CheckInBlockingMode(const Socket* sock);
+
+// Encode and send a message over a socket.
+// header: Request or Response header protobuf.
+// msg: Protobuf message to send. This message must be fully initialized.
+// deadline: Latest time allowed for receive to complete before timeout.
+Status SendFramedMessageBlocking(Socket* sock, const google::protobuf::MessageLite& header,
+    const google::protobuf::MessageLite& msg, const MonoTime& deadline);
+
+// Receive a full message frame from the server.
+// recv_buf: buffer to use for reading the data from the socket.
+// header: Request or Response header protobuf.
+// param_buf: Slice into recv_buf containing unparsed RPC param protobuf data.
+// deadline: Latest time allowed for receive to complete before timeout.
+Status ReceiveFramedMessageBlocking(Socket* sock, faststring* recv_buf,
+    google::protobuf::MessageLite* header, Slice* param_buf, const MonoTime& deadline);
+
+} // namespace rpc
+} // namespace kudu
+
+#endif  // KUDU_RPC_BLOCKING_OPS_H
diff --git a/be/src/kudu/rpc/client_negotiation.cc b/be/src/kudu/rpc/client_negotiation.cc
new file mode 100644
index 0000000..02175f6
--- /dev/null
+++ b/be/src/kudu/rpc/client_negotiation.cc
@@ -0,0 +1,853 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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.
+
+#include "kudu/rpc/client_negotiation.h"
+
+#include <cstdint>
+#include <cstring>
+#include <map>
+#include <memory>
+#include <ostream>
+#include <set>
+#include <string>
+
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+#include <gssapi/gssapi.h>
+#include <gssapi/gssapi_krb5.h>
+#include <sasl/sasl.h>
+
+#include "kudu/gutil/basictypes.h"
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/stl_util.h"
+#include "kudu/gutil/strings/join.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/rpc/blocking_ops.h"
+#include "kudu/rpc/constants.h"
+#include "kudu/rpc/messenger.h"
+#include "kudu/rpc/rpc_header.pb.h"
+#include "kudu/rpc/sasl_common.h"
+#include "kudu/rpc/sasl_helper.h"
+#include "kudu/rpc/serialization.h"
+#include "kudu/security/cert.h"
+#include "kudu/security/tls_context.h"
+#include "kudu/security/tls_handshake.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/net/socket.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/trace.h"
+
+using std::map;
+using std::set;
+using std::string;
+using std::unique_ptr;
+
+using strings::Substitute;
+
+DECLARE_bool(rpc_encrypt_loopback_connections);
+
+namespace kudu {
+namespace rpc {
+
+static int ClientNegotiationGetoptCb(ClientNegotiation* client_negotiation,
+                                     const char* plugin_name,
+                                     const char* option,
+                                     const char** result,
+                                     unsigned* len) {
+  return client_negotiation->GetOptionCb(plugin_name, option, result, len);
+}
+
+static int ClientNegotiationSimpleCb(ClientNegotiation* client_negotiation,
+                                     int id,
+                                     const char** result,
+                                     unsigned* len) {
+  return client_negotiation->SimpleCb(id, result, len);
+}
+
+static int ClientNegotiationSecretCb(sasl_conn_t* conn,
+                                     ClientNegotiation* client_negotiation,
+                                     int id,
+                                     sasl_secret_t** psecret) {
+  return client_negotiation->SecretCb(conn, id, psecret);
+}
+
+// Return an appropriately-typed Status object based on an ErrorStatusPB returned
+// from an Error RPC.
+// In case there is no relevant Status type, return a RuntimeError.
+static Status StatusFromRpcError(const ErrorStatusPB& error) {
+  DCHECK(error.IsInitialized()) << "Error status PB must be initialized";
+  if (PREDICT_FALSE(!error.has_code())) {
+    return Status::RuntimeError(error.message());
+  }
+  const string code_name = ErrorStatusPB::RpcErrorCodePB_Name(error.code());
+  switch (error.code()) {
+    case ErrorStatusPB_RpcErrorCodePB_FATAL_UNAUTHORIZED: // fall-through
+    case ErrorStatusPB_RpcErrorCodePB_FATAL_INVALID_AUTHENTICATION_TOKEN:
+      return Status::NotAuthorized(code_name, error.message());
+    case ErrorStatusPB_RpcErrorCodePB_ERROR_UNAVAILABLE:
+      return Status::ServiceUnavailable(code_name, error.message());
+    default:
+      return Status::RuntimeError(code_name, error.message());
+  }
+}
+
+ClientNegotiation::ClientNegotiation(unique_ptr<Socket> socket,
+                                     const security::TlsContext* tls_context,
+                                     boost::optional<security::SignedTokenPB> authn_token,
+                                     RpcEncryption encryption,
+                                     std::string sasl_proto_name)
+    : socket_(std::move(socket)),
+      helper_(SaslHelper::CLIENT),
+      tls_context_(tls_context),
+      encryption_(encryption),
+      tls_negotiated_(false),
+      authn_token_(std::move(authn_token)),
+      psecret_(nullptr, std::free),
+      negotiated_authn_(AuthenticationType::INVALID),
+      negotiated_mech_(SaslMechanism::INVALID),
+      sasl_proto_name_(std::move(sasl_proto_name)),
+      deadline_(MonoTime::Max()) {
+  callbacks_.push_back(SaslBuildCallback(SASL_CB_GETOPT,
+      reinterpret_cast<int (*)()>(&ClientNegotiationGetoptCb), this));
+  callbacks_.push_back(SaslBuildCallback(SASL_CB_AUTHNAME,
+      reinterpret_cast<int (*)()>(&ClientNegotiationSimpleCb), this));
+  callbacks_.push_back(SaslBuildCallback(SASL_CB_PASS,
+      reinterpret_cast<int (*)()>(&ClientNegotiationSecretCb), this));
+  callbacks_.push_back(SaslBuildCallback(SASL_CB_LIST_END, nullptr, nullptr));
+  DCHECK(socket_);
+  DCHECK(tls_context_);
+}
+
+Status ClientNegotiation::EnablePlain(const string& user, const string& pass) {
+  RETURN_NOT_OK(helper_.EnablePlain());
+  plain_auth_user_ = user;
+  plain_pass_ = pass;
+  return Status::OK();
+}
+
+Status ClientNegotiation::EnableGSSAPI() {
+  return helper_.EnableGSSAPI();
+}
+
+SaslMechanism::Type ClientNegotiation::negotiated_mechanism() const {
+  return negotiated_mech_;
+}
+
+void ClientNegotiation::set_server_fqdn(const string& domain_name) {
+  helper_.set_server_fqdn(domain_name);
+}
+
+void ClientNegotiation::set_deadline(const MonoTime& deadline) {
+  deadline_ = deadline;
+}
+
+Status ClientNegotiation::Negotiate(unique_ptr<ErrorStatusPB>* rpc_error) {
+  TRACE("Beginning negotiation");
+
+  // Ensure we can use blocking calls on the socket during negotiation.
+  RETURN_NOT_OK(CheckInBlockingMode(socket_.get()));
+
+  // Step 1: send the connection header.
+  RETURN_NOT_OK(SendConnectionHeader());
+
+  faststring recv_buf;
+
+  { // Step 2: send and receive the NEGOTIATE step messages.
+    RETURN_NOT_OK(SendNegotiate());
+    NegotiatePB response;
+    RETURN_NOT_OK(RecvNegotiatePB(&response, &recv_buf, rpc_error));
+    RETURN_NOT_OK(HandleNegotiate(response));
+    TRACE("Negotiated authn=$0", AuthenticationTypeToString(negotiated_authn_));
+  }
+
+  // Step 3: if both ends support TLS, do a TLS handshake.
+  // TODO(KUDU-1921): allow the client to require TLS.
+  if (encryption_ != RpcEncryption::DISABLED &&
+      ContainsKey(server_features_, TLS)) {
+    RETURN_NOT_OK(tls_context_->InitiateHandshake(security::TlsHandshakeType::CLIENT,
+                                                  &tls_handshake_));
+
+    if (negotiated_authn_ == AuthenticationType::SASL) {
+      // When using SASL authentication, verifying the server's certificate is
+      // not necessary. This allows the client to still use TLS encryption for
+      // connections to servers which only have a self-signed certificate.
+      tls_handshake_.set_verification_mode(security::TlsVerificationMode::VERIFY_NONE);
+    }
+
+    // To initiate the TLS handshake, we pretend as if the server sent us an
+    // empty TLS_HANDSHAKE token.
+    NegotiatePB initial;
+    initial.set_step(NegotiatePB::TLS_HANDSHAKE);
+    initial.set_tls_handshake("");
+    Status s = HandleTlsHandshake(initial);
+
+    while (s.IsIncomplete()) {
+      NegotiatePB response;
+      RETURN_NOT_OK(RecvNegotiatePB(&response, &recv_buf, rpc_error));
+      s = HandleTlsHandshake(response);
+    }
+    RETURN_NOT_OK(s);
+    tls_negotiated_ = true;
+  }
+
+  // Step 4: Authentication
+  switch (negotiated_authn_) {
+    case AuthenticationType::SASL:
+      RETURN_NOT_OK(AuthenticateBySasl(&recv_buf, rpc_error));
+      break;
+    case AuthenticationType::TOKEN:
+      RETURN_NOT_OK(AuthenticateByToken(&recv_buf, rpc_error));
+      break;
+    case AuthenticationType::CERTIFICATE:
+      // The TLS handshake has already authenticated the server.
+      break;
+    case AuthenticationType::INVALID: LOG(FATAL) << "unreachable";
+  }
+
+  // Step 5: Send connection context.
+  RETURN_NOT_OK(SendConnectionContext());
+
+  TRACE("Negotiation successful");
+  return Status::OK();
+}
+
+Status ClientNegotiation::SendNegotiatePB(const NegotiatePB& msg) {
+  RequestHeader header;
+  header.set_call_id(kNegotiateCallId);
+
+  DCHECK(socket_);
+  DCHECK(msg.IsInitialized()) << "message must be initialized";
+  DCHECK(msg.has_step()) << "message must have a step";
+
+  TRACE("Sending $0 NegotiatePB request", NegotiatePB::NegotiateStep_Name(msg.step()));
+  return SendFramedMessageBlocking(socket(), header, msg, deadline_);
+}
+
+Status ClientNegotiation::RecvNegotiatePB(NegotiatePB* msg,
+                                          faststring* buffer,
+                                          unique_ptr<ErrorStatusPB>* rpc_error) {
+  ResponseHeader header;
+  Slice param_buf;
+  RETURN_NOT_OK(ReceiveFramedMessageBlocking(socket(), buffer, &header, &param_buf, deadline_));
+  RETURN_NOT_OK(helper_.CheckNegotiateCallId(header.call_id()));
+
+  if (header.is_error()) {
+    return ParseError(param_buf, rpc_error);
+  }
+
+  RETURN_NOT_OK(helper_.ParseNegotiatePB(param_buf, msg));
+  TRACE("Received $0 NegotiatePB response", NegotiatePB::NegotiateStep_Name(msg->step()));
+  return Status::OK();
+}
+
+Status ClientNegotiation::ParseError(const Slice& err_data,
+                                     unique_ptr<ErrorStatusPB>* rpc_error) {
+  unique_ptr<ErrorStatusPB> error(new ErrorStatusPB);
+  if (!error->ParseFromArray(err_data.data(), err_data.size())) {
+    return Status::IOError("invalid error response, missing fields",
+                           error->InitializationErrorString());
+  }
+  Status s = StatusFromRpcError(*error);
+  TRACE("Received error response from server: $0", s.ToString());
+
+  if (rpc_error) {
+    rpc_error->swap(error);
+  }
+  return s;
+}
+
+Status ClientNegotiation::SendConnectionHeader() {
+  const uint8_t buflen = kMagicNumberLength + kHeaderFlagsLength;
+  uint8_t buf[buflen];
+  serialization::SerializeConnHeader(buf);
+  size_t nsent;
+  return socket()->BlockingWrite(buf, buflen, &nsent, deadline_);
+}
+
+Status ClientNegotiation::InitSaslClient() {
+  // TODO(KUDU-1922): consider setting SASL_SUCCESS_DATA
+  unsigned flags = 0;
+
+  sasl_conn_t* sasl_conn = nullptr;
+  RETURN_NOT_OK_PREPEND(WrapSaslCall(nullptr /* no conn */, [&]() {
+      return sasl_client_new(
+          sasl_proto_name_.c_str(),     // Registered name of the service using SASL. Required.
+          helper_.server_fqdn(),        // The fully qualified domain name of the remote server.
+          nullptr,                      // Local and remote IP address strings. (we don't use
+          nullptr,                      // any mechanisms which require this info.)
+          &callbacks_[0],               // Connection-specific callbacks.
+          flags,
+          &sasl_conn);
+    }), Substitute("unable to create new SASL $0 client", sasl_proto_name_));
+  sasl_conn_.reset(sasl_conn);
+  return Status::OK();
+}
+
+Status ClientNegotiation::SendNegotiate() {
+  NegotiatePB msg;
+  msg.set_step(NegotiatePB::NEGOTIATE);
+
+  // Advertise our supported features.
+  client_features_ = kSupportedClientRpcFeatureFlags;
+
+  if (encryption_ != RpcEncryption::DISABLED) {
+    client_features_.insert(TLS);
+    // If the remote peer is local, then we allow using TLS for authentication
+    // without encryption or integrity.
+    if (socket_->IsLoopbackConnection() && !FLAGS_rpc_encrypt_loopback_connections) {
+      client_features_.insert(TLS_AUTHENTICATION_ONLY);
+    }
+  }
+
+  for (RpcFeatureFlag feature : client_features_) {
+    msg.add_supported_features(feature);
+  }
+
+  if (!helper_.EnabledMechs().empty()) {
+    msg.add_authn_types()->mutable_sasl();
+  }
+  if (tls_context_->has_signed_cert() && !tls_context_->is_external_cert()) {
+    // We only provide authenticated TLS if the certificates are generated
+    // by the internal CA.
+    msg.add_authn_types()->mutable_certificate();
+  }
+  if (authn_token_ && tls_context_->has_trusted_cert()) {
+    // TODO(KUDU-1924): check that the authn token is not expired. Can this be done
+    // reliably on clients?
+    msg.add_authn_types()->mutable_token();
+  }
+
+  if (PREDICT_FALSE(msg.authn_types().empty())) {
+    return Status::NotAuthorized("client is not configured with an authentication type");
+  }
+
+  RETURN_NOT_OK(SendNegotiatePB(msg));
+  return Status::OK();
+}
+
+Status ClientNegotiation::HandleNegotiate(const NegotiatePB& response) {
+  if (PREDICT_FALSE(response.step() != NegotiatePB::NEGOTIATE)) {
+    return Status::NotAuthorized("expected NEGOTIATE step",
+                                 NegotiatePB::NegotiateStep_Name(response.step()));
+  }
+  TRACE("Received NEGOTIATE response from server");
+
+  // Fill in the set of features supported by the server.
+  for (int flag : response.supported_features()) {
+    // We only add the features that our local build knows about.
+    RpcFeatureFlag feature_flag = RpcFeatureFlag_IsValid(flag) ?
+                                  static_cast<RpcFeatureFlag>(flag) : UNKNOWN;
+    if (feature_flag != UNKNOWN) {
+      server_features_.insert(feature_flag);
+    }
+  }
+
+  if (encryption_ == RpcEncryption::REQUIRED &&
+      !ContainsKey(server_features_, RpcFeatureFlag::TLS)) {
+    return Status::NotAuthorized("server does not support required TLS encryption");
+  }
+
+  // Get the authentication type which the server would like to use.
+  DCHECK_LE(response.authn_types().size(), 1);
+  if (response.authn_types().empty()) {
+    // If the server doesn't send back an authentication type, default to SASL
+    // in order to maintain backwards compatibility.
+    negotiated_authn_ = AuthenticationType::SASL;
+  } else {
+    const auto& authn_type = response.authn_types(0);
+    switch (authn_type.type_case()) {
+      case AuthenticationTypePB::kSasl:
+        negotiated_authn_ = AuthenticationType::SASL;
+        break;
+      case AuthenticationTypePB::kToken:
+        // TODO(todd): we should also be checking tls_context_->has_trusted_cert()
+        // here to match the original logic we used to advertise TOKEN support,
+        // or perhaps just check explicitly whether we advertised TOKEN.
+        if (!authn_token_) {
+          return Status::RuntimeError(
+              "server chose token authentication, but client has no token");
+        }
+        negotiated_authn_ = AuthenticationType::TOKEN;
+        return Status::OK();
+      case AuthenticationTypePB::kCertificate:
+        if (!tls_context_->has_signed_cert()) {
+          return Status::RuntimeError(
+              "server chose certificate authentication, but client has no certificate");
+        }
+        negotiated_authn_ = AuthenticationType::CERTIFICATE;
+        return Status::OK();
+      case AuthenticationTypePB::TYPE_NOT_SET:
+        return Status::RuntimeError("server chose an unknown authentication type");
+    }
+  }
+
+  DCHECK_EQ(negotiated_authn_, AuthenticationType::SASL);
+
+  // Build a map of the SASL mechanisms offered by the server.
+  set<SaslMechanism::Type> client_mechs(helper_.EnabledMechs());
+  set<SaslMechanism::Type> server_mechs;
+  for (const NegotiatePB::SaslMechanism& sasl_mech : response.sasl_mechanisms()) {
+    auto mech = SaslMechanism::value_of(sasl_mech.mechanism());
+    if (mech == SaslMechanism::INVALID) {
+      continue;
+    }
+    server_mechs.insert(mech);
+  }
+
+  // Determine which SASL mechanism to use for authenticating the connection.
+  // We pick the most preferred mechanism which is supported by both parties.
+  // The preference list in order of most to least preferred:
+  //  * GSSAPI
+  //  * PLAIN
+  //
+  // TODO(KUDU-1921): allow the client to require authentication.
+  if (ContainsKey(client_mechs, SaslMechanism::GSSAPI) &&
+      ContainsKey(server_mechs, SaslMechanism::GSSAPI)) {
+
+    // Check that the client has local Kerberos credentials, and if not fall
+    // back to an alternate mechanism.
+    Status s = CheckGSSAPI();
+    if (s.ok()) {
+      negotiated_mech_ = SaslMechanism::GSSAPI;
+      return Status::OK();
+    }
+
+    TRACE("Kerberos authentication credentials are not available: $0", s.ToString());
+    client_mechs.erase(SaslMechanism::GSSAPI);
+  }
+
+  if (ContainsKey(client_mechs, SaslMechanism::PLAIN) &&
+      ContainsKey(server_mechs, SaslMechanism::PLAIN)) {
+    negotiated_mech_ = SaslMechanism::PLAIN;
+    return Status::OK();
+  }
+
+  // There are no mechanisms in common.
+  if (ContainsKey(server_mechs, SaslMechanism::GSSAPI) &&
+      !ContainsKey(client_mechs, SaslMechanism::GSSAPI)) {
+    return Status::NotAuthorized("server requires authentication, "
+                                  "but client does not have Kerberos credentials available");
+  }
+  if (!ContainsKey(server_mechs, SaslMechanism::GSSAPI) &&
+      ContainsKey(client_mechs, SaslMechanism::GSSAPI)) {
+    return Status::NotAuthorized("client requires authentication, "
+                                  "but server does not have Kerberos enabled");
+  }
+  string msg = Substitute("client/server supported SASL mechanism mismatch; "
+                          "client mechanisms: [$0], server mechanisms: [$1]",
+                          JoinMapped(client_mechs, SaslMechanism::name_of, ", "),
+                          JoinMapped(server_mechs, SaslMechanism::name_of, ", "));
+
+  // For now, there should never be a SASL mechanism mismatch that isn't due
+  // to one of the sides requiring Kerberos and the other not having it, so
+  // lets sanity check that.
+  DCHECK(STLSetIntersection(client_mechs, server_mechs).empty()) << msg;
+  return Status::NotAuthorized(msg);
+}
+
+Status ClientNegotiation::SendTlsHandshake(string tls_token) {
+  TRACE("Sending TLS_HANDSHAKE message to server");
+  NegotiatePB msg;
+  msg.set_step(NegotiatePB::TLS_HANDSHAKE);
+  msg.mutable_tls_handshake()->swap(tls_token);
+  return SendNegotiatePB(msg);
+}
+
+Status ClientNegotiation::HandleTlsHandshake(const NegotiatePB& response) {
+  if (PREDICT_FALSE(response.step() != NegotiatePB::TLS_HANDSHAKE)) {
+    return Status::NotAuthorized("expected TLS_HANDSHAKE step",
+                                 NegotiatePB::NegotiateStep_Name(response.step()));
+  }
+  TRACE("Received TLS_HANDSHAKE response from server");
+
+  if (PREDICT_FALSE(!response.has_tls_handshake())) {
+    return Status::NotAuthorized("No TLS handshake token in TLS_HANDSHAKE response from server");
+  }
+
+  string token;
+  Status s = tls_handshake_.Continue(response.tls_handshake(), &token);
+  if (s.IsIncomplete()) {
+    // Another roundtrip is required to complete the handshake.
+    RETURN_NOT_OK(SendTlsHandshake(std::move(token)));
+  }
+
+  // Check that the handshake step didn't produce an error. Will also propagate
+  // an Incomplete status.
+  RETURN_NOT_OK(s);
+
+  // TLS handshake is finished.
+  if (ContainsKey(server_features_, TLS_AUTHENTICATION_ONLY) &&
+      ContainsKey(client_features_, TLS_AUTHENTICATION_ONLY)) {
+    TRACE("Negotiated auth-only $0 with cipher $1",
+          tls_handshake_.GetProtocol(), tls_handshake_.GetCipherDescription());
+    return tls_handshake_.FinishNoWrap(*socket_);
+  }
+
+  TRACE("Negotiated $0 with cipher $1",
+        tls_handshake_.GetProtocol(), tls_handshake_.GetCipherDescription());
+  return tls_handshake_.Finish(&socket_);
+}
+
+Status ClientNegotiation::AuthenticateBySasl(faststring* recv_buf,
+                                             unique_ptr<ErrorStatusPB>* rpc_error) {
+  RETURN_NOT_OK(InitSaslClient());
+  Status s = SendSaslInitiate();
+
+  // HandleSasl[Initiate, Challenge] return incomplete if an additional
+  // challenge step is required, or OK if a SASL_SUCCESS message is expected.
+  while (s.IsIncomplete()) {
+    NegotiatePB challenge;
+    RETURN_NOT_OK(RecvNegotiatePB(&challenge, recv_buf, rpc_error));
+    s = HandleSaslChallenge(challenge);
+  }
+
+  // Propagate failure from SendSaslInitiate or HandleSaslChallenge.
+  RETURN_NOT_OK(s);
+
+  // Server challenges are over; we now expect the success message.
+  NegotiatePB success;
+  RETURN_NOT_OK(RecvNegotiatePB(&success, recv_buf, rpc_error));
+  return HandleSaslSuccess(success);
+}
+
+Status ClientNegotiation::AuthenticateByToken(faststring* recv_buf,
+                                              unique_ptr<ErrorStatusPB>* rpc_error) {
+  // Sanity check that TLS has been negotiated. Sending the token on an
+  // unencrypted channel is a big no-no.
+  CHECK(tls_negotiated_);
+
+  // Send the token to the server.
+  NegotiatePB pb;
+  pb.set_step(NegotiatePB::TOKEN_EXCHANGE);
+  *pb.mutable_authn_token() = std::move(*authn_token_);
+  RETURN_NOT_OK(SendNegotiatePB(pb));
+  pb.Clear();
+
+  // Check that the server responds with a non-error TOKEN_EXCHANGE message.
+  RETURN_NOT_OK(RecvNegotiatePB(&pb, recv_buf, rpc_error));
+  if (pb.step() != NegotiatePB::TOKEN_EXCHANGE) {
+    return Status::NotAuthorized("expected TOKEN_EXCHANGE step",
+                                 NegotiatePB::NegotiateStep_Name(pb.step()));
+  }
+
+  return Status::OK();
+}
+
+Status ClientNegotiation::SendSaslInitiate() {
+  TRACE("Initiating SASL $0 handshake", SaslMechanism::name_of(negotiated_mech_));
+
+  // At this point we've already chosen the SASL mechanism to use
+  // (negotiated_mech_), but we need to let the SASL library know. SASL likes to
+  // choose the mechanism from among a list of possible options, so we simply
+  // provide it one option, and then check that it picks that option.
+
+  const char* init_msg = nullptr;
+  unsigned init_msg_len = 0;
+  const char* negotiated_mech = nullptr;
+
+  /* select a mechanism for a connection
+   *  mechlist      -- mechanisms server has available (punctuation ignored)
+   * output:
+   *  prompt_need   -- on SASL_INTERACT, list of prompts needed to continue
+   *  clientout     -- the initial client response to send to the server
+   *  mech          -- set to mechanism name
+   *
+   * Returns:
+   *  SASL_OK       -- success
+   *  SASL_CONTINUE -- negotiation required
+   *  SASL_NOMEM    -- not enough memory
+   *  SASL_NOMECH   -- no mechanism meets requested properties
+   *  SASL_INTERACT -- user interaction needed to fill in prompt_need list
+   */
+  TRACE("Calling sasl_client_start()");
+  const Status s = WrapSaslCall(sasl_conn_.get(), [&]() {
+      return sasl_client_start(
+          sasl_conn_.get(),                         // The SASL connection context created by init()
+          SaslMechanism::name_of(negotiated_mech_), // The list of mechanisms to negotiate.
+          nullptr,                                  // Disables INTERACT return if NULL.
+          &init_msg,                                // Filled in on success.
+          &init_msg_len,                            // Filled in on success.
+          &negotiated_mech);                        // Filled in on success.
+  });
+
+  if (PREDICT_FALSE(!s.IsIncomplete() && !s.ok())) {
+    return s;
+  }
+
+  // Check that the SASL library is using the mechanism that we picked.
+  DCHECK_EQ(SaslMechanism::value_of(negotiated_mech), negotiated_mech_);
+
+  // If the negotiated mechanism is GSSAPI (Kerberos), configure SASL to use
+  // integrity protection so that the channel bindings and nonce can be
+  // verified.
+  if (negotiated_mech_ == SaslMechanism::GSSAPI) {
+    RETURN_NOT_OK(EnableProtection(sasl_conn_.get(), SaslProtection::kIntegrity));
+  }
+
+  NegotiatePB msg;
+  msg.set_step(NegotiatePB::SASL_INITIATE);
+  msg.mutable_token()->assign(init_msg, init_msg_len);
+  msg.add_sasl_mechanisms()->set_mechanism(negotiated_mech);
+  RETURN_NOT_OK(SendNegotiatePB(msg));
+  return s;
+}
+
+Status ClientNegotiation::SendSaslResponse(const char* resp_msg, unsigned resp_msg_len) {
+  NegotiatePB reply;
+  reply.set_step(NegotiatePB::SASL_RESPONSE);
+  reply.mutable_token()->assign(resp_msg, resp_msg_len);
+  return SendNegotiatePB(reply);
+}
+
+Status ClientNegotiation::HandleSaslChallenge(const NegotiatePB& response) {
+  if (PREDICT_FALSE(response.step() != NegotiatePB::SASL_CHALLENGE)) {
+    return Status::NotAuthorized("expected SASL_CHALLENGE step",
+                                 NegotiatePB::NegotiateStep_Name(response.step()));
+  }
+  TRACE("Received SASL_CHALLENGE response from server");
+  if (PREDICT_FALSE(!response.has_token())) {
+    return Status::NotAuthorized("no token in SASL_CHALLENGE response from server");
+  }
+
+  const char* out = nullptr;
+  unsigned out_len = 0;
+  const Status s = DoSaslStep(response.token(), &out, &out_len);
+  if (PREDICT_FALSE(!s.IsIncomplete() && !s.ok())) {
+    return s;
+  }
+
+  RETURN_NOT_OK(SendSaslResponse(out, out_len));
+  return s;
+}
+
+Status ClientNegotiation::HandleSaslSuccess(const NegotiatePB& response) {
+  if (PREDICT_FALSE(response.step() != NegotiatePB::SASL_SUCCESS)) {
+    return Status::NotAuthorized("expected SASL_SUCCESS step",
+                                 NegotiatePB::NegotiateStep_Name(response.step()));
+  }
+  TRACE("Received SASL_SUCCESS response from server");
+
+  if (negotiated_mech_ == SaslMechanism::GSSAPI) {
+    if (response.has_nonce()) {
+      // Grab the nonce from the server, if it has sent one. We'll send it back
+      // later with SASL integrity protection as part of the connection context.
+      nonce_ = response.nonce();
+    }
+
+    if (tls_negotiated_) {
+      // Check the channel bindings provided by the server against the expected channel bindings.
+      if (!response.has_channel_bindings()) {
+        return Status::NotAuthorized("no channel bindings provided by server");
+      }
+
+      security::Cert cert;
+      RETURN_NOT_OK(tls_handshake_.GetRemoteCert(&cert));
+
+      string expected_channel_bindings;
+      RETURN_NOT_OK_PREPEND(cert.GetServerEndPointChannelBindings(&expected_channel_bindings),
+                            "failed to generate channel bindings");
+
+      Slice received_channel_bindings;
+      RETURN_NOT_OK_PREPEND(SaslDecode(sasl_conn_.get(),
+                                       response.channel_bindings(),
+                                       &received_channel_bindings),
+                            "failed to decode channel bindings");
+
+      if (expected_channel_bindings != received_channel_bindings) {
+        Sockaddr addr;
+        ignore_result(socket_->GetPeerAddress(&addr));
+
+        LOG(WARNING) << "Received invalid channel bindings from server "
+                    << addr.ToString()
+                    << ", this could indicate an active network man-in-the-middle";
+        return Status::NotAuthorized("channel bindings do not match");
+      }
+    }
+  }
+
+  return Status::OK();
+}
+
+Status ClientNegotiation::DoSaslStep(const string& in, const char** out, unsigned* out_len) {
+  TRACE("Calling sasl_client_step()");
+
+  return WrapSaslCall(sasl_conn_.get(), [&]() {
+      return sasl_client_step(sasl_conn_.get(), in.c_str(), in.length(), nullptr, out, out_len);
+  });
+}
+
+Status ClientNegotiation::SendConnectionContext() {
+  TRACE("Sending connection context");
+  RequestHeader header;
+  header.set_call_id(kConnectionContextCallId);
+
+  ConnectionContextPB conn_context;
+  // This field is deprecated but used by servers <Kudu 1.1. Newer server versions ignore
+  // this and use the SASL-provided username instead.
+  conn_context.mutable_deprecated_user_info()->set_real_user(
+      plain_auth_user_.empty() ? "cpp-client" : plain_auth_user_);
+
+  if (nonce_) {
+    // Reply with the SASL-protected nonce. We only set the nonce when using SASL GSSAPI.
+    Slice ciphertext;
+    RETURN_NOT_OK(SaslEncode(sasl_conn_.get(), *nonce_, &ciphertext));
+    *conn_context.mutable_encoded_nonce() = ciphertext.ToString();
+  }
+
+  return SendFramedMessageBlocking(socket(), header, conn_context, deadline_);
+}
+
+int ClientNegotiation::GetOptionCb(const char* plugin_name, const char* option,
+                            const char** result, unsigned* len) {
+  return helper_.GetOptionCb(plugin_name, option, result, len);
+}
+
+// Used for PLAIN.
+// SASL callback for SASL_CB_USER, SASL_CB_AUTHNAME, SASL_CB_LANGUAGE
+int ClientNegotiation::SimpleCb(int id, const char** result, unsigned* len) {
+  if (PREDICT_FALSE(!helper_.IsPlainEnabled())) {
+    LOG(DFATAL) << "Simple callback called, but PLAIN auth is not enabled";
+    return SASL_FAIL;
+  }
+  if (PREDICT_FALSE(result == nullptr)) {
+    LOG(DFATAL) << "result outparam is NULL";
+    return SASL_BADPARAM;
+  }
+  switch (id) {
+    // TODO(unknown): Support impersonation?
+    // For impersonation, USER is the impersonated user, AUTHNAME is the "sudoer".
+    case SASL_CB_USER:
+      TRACE("callback for SASL_CB_USER");
+      *result = plain_auth_user_.c_str();
+      if (len != nullptr) *len = plain_auth_user_.length();
+      break;
+    case SASL_CB_AUTHNAME:
+      TRACE("callback for SASL_CB_AUTHNAME");
+      *result = plain_auth_user_.c_str();
+      if (len != nullptr) *len = plain_auth_user_.length();
+      break;
+    case SASL_CB_LANGUAGE:
+      LOG(DFATAL) << "Unable to handle SASL callback type SASL_CB_LANGUAGE"
+        << "(" << id << ")";
+      return SASL_BADPARAM;
+    default:
+      LOG(DFATAL) << "Unexpected SASL callback type: " << id;
+      return SASL_BADPARAM;
+  }
+
+  return SASL_OK;
+}
+
+// Used for PLAIN.
+// SASL callback for SASL_CB_PASS: User password.
+int ClientNegotiation::SecretCb(sasl_conn_t* conn, int id, sasl_secret_t** psecret) {
+  if (PREDICT_FALSE(!helper_.IsPlainEnabled())) {
+    LOG(DFATAL) << "Plain secret callback called, but PLAIN auth is not enabled";
+    return SASL_FAIL;
+  }
+  switch (id) {
+    case SASL_CB_PASS: {
+      if (!conn || !psecret) return SASL_BADPARAM;
+
+      size_t len = plain_pass_.length();
+      *psecret = reinterpret_cast<sasl_secret_t*>(malloc(sizeof(sasl_secret_t) + len));
+      if (!*psecret) {
+        return SASL_NOMEM;
+      }
+      psecret_.reset(*psecret);  // Ensure that we free() this structure later.
+      (*psecret)->len = len;
+      memcpy((*psecret)->data, plain_pass_.c_str(), len + 1);
+      break;
+    }
+    default:
+      LOG(DFATAL) << "Unexpected SASL callback type: " << id;
+      return SASL_BADPARAM;
+  }
+
+  return SASL_OK;
+}
+
+namespace {
+// Retrieve the GSSAPI error description for an error code and type.
+string gss_error_description(OM_uint32 code, int type) {
+  string description;
+  OM_uint32 message_context = 0;
+
+  do {
+    if (!description.empty()) {
+      description.append(": ");
+    }
+    OM_uint32 minor = 0;
+    gss_buffer_desc buf;
+    gss_display_status(&minor, code, type, GSS_C_NULL_OID, &message_context, &buf);
+    description.append(static_cast<const char*>(buf.value), buf.length);
+    gss_release_buffer(&minor, &buf);
+  } while (message_context != 0);
+
+  return description;
+}
+
+// Transforms a GSSAPI major and minor error code into a Kudu Status.
+Status check_gss_error(OM_uint32 major, OM_uint32 minor) {
+    if (GSS_ERROR(major)) {
+      return Status::NotAuthorized(gss_error_description(major, GSS_C_GSS_CODE),
+                                   gss_error_description(minor, GSS_C_MECH_CODE));
+    }
+    return Status::OK();
+}
+} // anonymous namespace
+
+Status ClientNegotiation::CheckGSSAPI() {
+  OM_uint32 major, minor;
+  gss_cred_id_t cred = GSS_C_NO_CREDENTIAL;
+
+  // Acquire the Kerberos credential. This will fail if the client does not have
+  // a Kerberos tgt ticket. In theory it should be sufficient to call
+  // gss_inquire_cred_by_mech, but that causes a memory leak on RHEL 7.
+  major = gss_acquire_cred(&minor,
+                           GSS_C_NO_NAME,
+                           GSS_C_INDEFINITE,
+                           const_cast<gss_OID_set>(gss_mech_set_krb5),
+                           GSS_C_INITIATE,
+                           &cred,
+                           nullptr,
+                           nullptr);
+  Status s = check_gss_error(major, minor);
+
+  // Inspect the Kerberos credential to determine if it is expired. The lifetime
+  // returned from gss_acquire_cred in the RHEL 6 version of krb5 is always 0,
+  // so it has to be done with a separate call to gss_inquire_cred. The lifetime
+  // holds the remaining validity of the tgt in seconds.
+  OM_uint32 lifetime;
+  if (s.ok()) {
+    major = gss_inquire_cred(&minor, cred, nullptr, &lifetime, nullptr, nullptr);
+    s = check_gss_error(major, minor);
+  }
+
+  // Release the credential even if gss_inquire_cred fails.
+  gss_release_cred(&minor, &cred);
+  RETURN_NOT_OK(s);
+
+  if (lifetime == 0) {
+    return Status::NotAuthorized("Kerberos ticket expired");
+  }
+  return Status::OK();
+}
+
+} // namespace rpc
+} // namespace kudu
diff --git a/be/src/kudu/rpc/client_negotiation.h b/be/src/kudu/rpc/client_negotiation.h
new file mode 100644
index 0000000..06fb2b8
--- /dev/null
+++ b/be/src/kudu/rpc/client_negotiation.h
@@ -0,0 +1,263 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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.
+
+#pragma once
+
+#include <cstdlib>
+#include <memory>
+#include <set>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include <boost/optional/optional.hpp>
+#include <glog/logging.h>
+#include <sasl/sasl.h>
+
+#include "kudu/rpc/messenger.h"
+#include "kudu/rpc/negotiation.h"
+#include "kudu/rpc/rpc_header.pb.h"
+#include "kudu/rpc/sasl_common.h"
+#include "kudu/rpc/sasl_helper.h"
+#include "kudu/security/security_flags.h"
+#include "kudu/security/tls_handshake.h"
+#include "kudu/security/token.pb.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/net/socket.h"
+#include "kudu/gutil/port.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+class Slice;
+class faststring;
+
+namespace security {
+class TlsContext;
+}
+
+namespace rpc {
+
+// Class for doing KRPC negotiation with a remote server over a bidirectional socket.
+// Operations on this class are NOT thread-safe.
+class ClientNegotiation {
+ public:
+  // Creates a new client negotiation instance, taking ownership of the
+  // provided socket. After completing the negotiation process by setting the
+  // desired options and calling Negotiate(), the socket can be retrieved with
+  // 'release_socket'.
+  //
+  // The provided TlsContext must outlive this negotiation instance.
+  ClientNegotiation(std::unique_ptr<Socket> socket,
+                    const security::TlsContext* tls_context,
+                    boost::optional<security::SignedTokenPB> authn_token,
+                    RpcEncryption encryption,
+                    std::string sasl_proto_name);
+
+  // Enable PLAIN authentication.
+  // Must be called before Negotiate().
+  Status EnablePlain(const std::string& user,
+                     const std::string& pass);
+
+  // Enable GSSAPI authentication.
+  // Must be called before Negotiate().
+  Status EnableGSSAPI();
+
+  // Returns mechanism negotiated by this connection.
+  // Must be called after Negotiate().
+  SaslMechanism::Type negotiated_mechanism() const;
+
+  // Returns the negotiated authentication type for the connection.
+  // Must be called after Negotiate().
+  AuthenticationType negotiated_authn() const {
+    DCHECK_NE(negotiated_authn_, AuthenticationType::INVALID);
+    return negotiated_authn_;
+  }
+
+  // Returns true if TLS was negotiated.
+  // Must be called after Negotiate().
+  bool tls_negotiated() const {
+    return tls_negotiated_;
+  }
+
+  // Returns the set of RPC system features supported by the remote server.
+  // Must be called before Negotiate().
+  std::set<RpcFeatureFlag> server_features() const {
+    return server_features_;
+  }
+
+  // Returns the set of RPC system features supported by the remote server.
+  // Must be called after Negotiate().
+  // Subsequent calls to this method or server_features() will return an empty set.
+  std::set<RpcFeatureFlag> take_server_features() {
+    return std::move(server_features_);
+  }
+
+  // Specify the fully-qualified domain name of the remote server.
+  // Must be called before Negotiate(). Required for some mechanisms.
+  void set_server_fqdn(const std::string& domain_name);
+
+  // Set deadline for connection negotiation.
+  void set_deadline(const MonoTime& deadline);
+
+  Socket* socket() { return socket_.get(); }
+
+  // Takes and returns the socket owned by this client negotiation. The caller
+  // will own the socket after this call, and the negotiation instance should no
+  // longer be used. Must be called after Negotiate(). Subsequent calls to this
+  // method or socket() will return a null pointer.
+  std::unique_ptr<Socket> release_socket() { return std::move(socket_); }
+
+  // Negotiate with the remote server. Should only be called once per
+  // ClientNegotiation and socket instance, after all options have been set.
+  //
+  // Returns OK on success, otherwise may return NotAuthorized, NotSupported, or
+  // another non-OK status.
+  Status Negotiate(std::unique_ptr<ErrorStatusPB>* rpc_error = nullptr);
+
+  // SASL callback for plugin options, supported mechanisms, etc.
+  // Returns SASL_FAIL if the option is not handled, which does not fail the handshake.
+  int GetOptionCb(const char* plugin_name, const char* option,
+                  const char** result, unsigned* len);
+
+  // SASL callback for SASL_CB_USER, SASL_CB_AUTHNAME, SASL_CB_LANGUAGE
+  int SimpleCb(int id, const char** result, unsigned* len);
+
+  // SASL callback for SASL_CB_PASS
+  int SecretCb(sasl_conn_t* conn, int id, sasl_secret_t** psecret);
+
+  // Check that GSSAPI/Kerberos credentials are available.
+  static Status CheckGSSAPI() WARN_UNUSED_RESULT;
+
+ private:
+
+  // Encode and send the specified negotiate request message to the server.
+  Status SendNegotiatePB(const NegotiatePB& msg) WARN_UNUSED_RESULT;
+
+  // Receive a negotiate response message from the server, deserializing it into 'msg'.
+  // Validates that the response is not an error.
+  Status RecvNegotiatePB(NegotiatePB* msg,
+                         faststring* buffer,
+                         std::unique_ptr<ErrorStatusPB>* rpc_error) WARN_UNUSED_RESULT;
+
+  // Parse error status message from raw bytes of an ErrorStatusPB.
+  Status ParseError(const Slice& err_data,
+                    std::unique_ptr<ErrorStatusPB>* rpc_error) WARN_UNUSED_RESULT;
+
+  Status SendConnectionHeader() WARN_UNUSED_RESULT;
+
+  // Initialize the SASL client negotiation instance.
+  Status InitSaslClient() WARN_UNUSED_RESULT;
+
+  // Send a NEGOTIATE step message to the server.
+  Status SendNegotiate() WARN_UNUSED_RESULT;
+
+  // Handle NEGOTIATE step response from the server.
+  Status HandleNegotiate(const NegotiatePB& response) WARN_UNUSED_RESULT;
+
+  // Send a TLS_HANDSHAKE request message to the server with the provided token.
+  Status SendTlsHandshake(std::string tls_token) WARN_UNUSED_RESULT;
+
+  // Handle a TLS_HANDSHAKE response message from the server.
+  Status HandleTlsHandshake(const NegotiatePB& response) WARN_UNUSED_RESULT;
+
+  // Authenticate to the server using SASL.
+  // 'recv_buf' allows a receive buffer to be reused.
+  Status AuthenticateBySasl(faststring* recv_buf,
+                            std::unique_ptr<ErrorStatusPB>* rpc_error) WARN_UNUSED_RESULT;
+
+  // Authenticate to the server using a token.
+  // 'recv_buf' allows a receive buffer to be reused.
+  Status AuthenticateByToken(faststring* recv_buf,
+                             std::unique_ptr<ErrorStatusPB> *rpc_error) WARN_UNUSED_RESULT;
+
+  // Send an SASL_INITIATE message to the server.
+  // Returns:
+  //  Status::OK if the SASL_SUCCESS message is expected next.
+  //  Status::Incomplete if the SASL_CHALLENGE message is expected next.
+  //  Any other status indicates an error.
+  Status SendSaslInitiate() WARN_UNUSED_RESULT;
+
+  // Send a SASL_RESPONSE message to the server.
+  Status SendSaslResponse(const char* resp_msg, unsigned resp_msg_len) WARN_UNUSED_RESULT;
+
+  // Handle case when server sends SASL_CHALLENGE response.
+  // Returns:
+  //  Status::OK if a SASL_SUCCESS message is expected next.
+  //  Status::Incomplete if another SASL_CHALLENGE message is expected.
+  //  Any other status indicates an error.
+  Status HandleSaslChallenge(const NegotiatePB& response) WARN_UNUSED_RESULT;
+
+  // Handle case when server sends SASL_SUCCESS response.
+  Status HandleSaslSuccess(const NegotiatePB& response) WARN_UNUSED_RESULT;
+
+  // Perform a client-side step of the SASL negotiation.
+  // Input is what came from the server. Output is what we will send back to the server.
+  // Returns:
+  //   Status::OK if sasl_client_step returns SASL_OK.
+  //   Status::Incomplete if sasl_client_step returns SASL_CONTINUE
+  // otherwise returns an appropriate error status.
+  Status DoSaslStep(const std::string& in, const char** out, unsigned* out_len) WARN_UNUSED_RESULT;
+
+  Status SendConnectionContext() WARN_UNUSED_RESULT;
+
+  // The socket to the remote server.
+  std::unique_ptr<Socket> socket_;
+
+  // SASL state.
+  std::vector<sasl_callback_t> callbacks_;
+  std::unique_ptr<sasl_conn_t, SaslDeleter> sasl_conn_;
+  SaslHelper helper_;
+  boost::optional<std::string> nonce_;
+
+  // TLS state.
+  const security::TlsContext* tls_context_;
+  security::TlsHandshake tls_handshake_;
+  const RpcEncryption encryption_;
+  bool tls_negotiated_;
+
+  // TSK state.
+  boost::optional<security::SignedTokenPB> authn_token_;
+
+  // Authentication state.
+  std::string plain_auth_user_;
+  std::string plain_pass_;
+  std::unique_ptr<sasl_secret_t, decltype(std::free)*> psecret_;
+
+  // The set of features advertised by the client. Filled in when we send
+  // the first message. This is not necessarily constant since some features
+  // may be dynamically enabled.
+  std::set<RpcFeatureFlag> client_features_;
+
+  // The set of features supported by the server. Filled in during negotiation.
+  std::set<RpcFeatureFlag> server_features_;
+
+  // The authentication type. Filled in during negotiation.
+  AuthenticationType negotiated_authn_;
+
+  // The SASL mechanism used by the connection. Filled in during negotiation.
+  SaslMechanism::Type negotiated_mech_;
+
+  // The SASL protocol name that is used for the SASL negotiation.
+  const std::string sasl_proto_name_;
+
+  // Negotiation timeout deadline.
+  MonoTime deadline_;
+};
+
+} // namespace rpc
+} // namespace kudu
diff --git a/be/src/kudu/rpc/connection.cc b/be/src/kudu/rpc/connection.cc
new file mode 100644
index 0000000..1632dd3
--- /dev/null
+++ b/be/src/kudu/rpc/connection.cc
@@ -0,0 +1,767 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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.
+
+#include "kudu/rpc/connection.h"
+
+#include <algorithm>
+#include <cerrno>
+#include <iostream>
+#include <memory>
+#include <set>
+#include <string>
+#include <type_traits>
+
+#include <boost/intrusive/detail/list_iterator.hpp>
+#include <boost/intrusive/list.hpp>
+#include <ev.h>
+#include <glog/logging.h>
+
+#include "kudu/gutil/map-util.h"
+#include "kudu/util/slice.h"
+#include "kudu/gutil/strings/human_readable.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/rpc/inbound_call.h"
+#include "kudu/rpc/messenger.h"
+#include "kudu/rpc/outbound_call.h"
+#include "kudu/rpc/reactor.h"
+#include "kudu/rpc/rpc_controller.h"
+#include "kudu/rpc/rpc_header.pb.h"
+#include "kudu/rpc/rpc_introspection.pb.h"
+#include "kudu/rpc/transfer.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/net/socket.h"
+#include "kudu/util/status.h"
+
+using std::includes;
+using std::set;
+using std::shared_ptr;
+using std::unique_ptr;
+using strings::Substitute;
+
+namespace kudu {
+namespace rpc {
+
+typedef OutboundCall::Phase Phase;
+
+///
+/// Connection
+///
+Connection::Connection(ReactorThread *reactor_thread,
+                       Sockaddr remote,
+                       unique_ptr<Socket> socket,
+                       Direction direction,
+                       CredentialsPolicy policy)
+    : reactor_thread_(reactor_thread),
+      remote_(remote),
+      socket_(std::move(socket)),
+      direction_(direction),
+      last_activity_time_(MonoTime::Now()),
+      is_epoll_registered_(false),
+      next_call_id_(1),
+      credentials_policy_(policy),
+      negotiation_complete_(false),
+      is_confidential_(false),
+      scheduled_for_shutdown_(false) {
+}
+
+Status Connection::SetNonBlocking(bool enabled) {
+  return socket_->SetNonBlocking(enabled);
+}
+
+void Connection::EpollRegister(ev::loop_ref& loop) {
+  DCHECK(reactor_thread_->IsCurrentThread());
+  DVLOG(4) << "Registering connection for epoll: " << ToString();
+  write_io_.set(loop);
+  write_io_.set(socket_->GetFd(), ev::WRITE);
+  write_io_.set<Connection, &Connection::WriteHandler>(this);
+  if (direction_ == CLIENT && negotiation_complete_) {
+    write_io_.start();
+  }
+  read_io_.set(loop);
+  read_io_.set(socket_->GetFd(), ev::READ);
+  read_io_.set<Connection, &Connection::ReadHandler>(this);
+  read_io_.start();
+  is_epoll_registered_ = true;
+}
+
+Connection::~Connection() {
+  // Must clear the outbound_transfers_ list before deleting.
+  CHECK(outbound_transfers_.begin() == outbound_transfers_.end());
+
+  // It's crucial that the connection is Shutdown first -- otherwise
+  // our destructor will end up calling read_io_.stop() and write_io_.stop()
+  // from a possibly non-reactor thread context. This can then make all
+  // hell break loose with libev.
+  CHECK(!is_epoll_registered_);
+}
+
+bool Connection::Idle() const {
+  DCHECK(reactor_thread_->IsCurrentThread());
+  // check if we're in the middle of receiving something
+  InboundTransfer *transfer = inbound_.get();
+  if (transfer && (transfer->TransferStarted())) {
+    return false;
+  }
+  // check if we still need to send something
+  if (!outbound_transfers_.empty()) {
+    return false;
+  }
+  // can't kill a connection if calls are waiting response
+  if (!awaiting_response_.empty()) {
+    return false;
+  }
+
+  if (!calls_being_handled_.empty()) {
+    return false;
+  }
+
+  // We are not idle if we are in the middle of connection negotiation.
+  if (!negotiation_complete_) {
+    return false;
+  }
+
+  return true;
+}
+
+void Connection::Shutdown(const Status &status,
+                          unique_ptr<ErrorStatusPB> rpc_error) {
+  DCHECK(reactor_thread_->IsCurrentThread());
+  shutdown_status_ = status.CloneAndPrepend("RPC connection failed");
+
+  if (inbound_ && inbound_->TransferStarted()) {
+    double secs_since_active =
+        (reactor_thread_->cur_time() - last_activity_time_).ToSeconds();
+    LOG(WARNING) << "Shutting down " << ToString()
+                 << " with pending inbound data ("
+                 << inbound_->StatusAsString() << ", last active "
+                 << HumanReadableElapsedTime::ToShortString(secs_since_active)
+                 << " ago, status=" << status.ToString() << ")";
+  }
+
+  // Clear any calls which have been sent and were awaiting a response.
+  for (const car_map_t::value_type &v : awaiting_response_) {
+    CallAwaitingResponse *c = v.second;
+    if (c->call) {
+      // Make sure every awaiting call receives the error info, if any.
+      unique_ptr<ErrorStatusPB> error;
+      if (rpc_error) {
+        error.reset(new ErrorStatusPB(*rpc_error));
+      }
+      c->call->SetFailed(status,
+                         negotiation_complete_ ? Phase::REMOTE_CALL
+                                               : Phase::CONNECTION_NEGOTIATION,
+                         std::move(error));
+    }
+    // And we must return the CallAwaitingResponse to the pool
+    car_pool_.Destroy(c);
+  }
+  awaiting_response_.clear();
+
+  // Clear any outbound transfers.
+  while (!outbound_transfers_.empty()) {
+    OutboundTransfer *t = &outbound_transfers_.front();
+    outbound_transfers_.pop_front();
+    delete t;
+  }
+
+  read_io_.stop();
+  write_io_.stop();
+  is_epoll_registered_ = false;
+  if (socket_) {
+    WARN_NOT_OK(socket_->Close(), "Error closing socket");
+  }
+}
+
+void Connection::QueueOutbound(gscoped_ptr<OutboundTransfer> transfer) {
+  DCHECK(reactor_thread_->IsCurrentThread());
+
+  if (!shutdown_status_.ok()) {
+    // If we've already shut down, then we just need to abort the
+    // transfer rather than bothering to queue it.
+    transfer->Abort(shutdown_status_);
+    return;
+  }
+
+  DVLOG(3) << "Queueing transfer: " << transfer->HexDump();
+
+  outbound_transfers_.push_back(*transfer.release());
+
+  if (negotiation_complete_ && !write_io_.is_active()) {
+    // If we weren't currently in the middle of sending anything,
+    // then our write_io_ interest is stopped. Need to re-start it.
+    // Only do this after connection negotiation is done doing its work.
+    write_io_.start();
+  }
+}
+
+Connection::CallAwaitingResponse::~CallAwaitingResponse() {
+  DCHECK(conn->reactor_thread_->IsCurrentThread());
+}
+
+void Connection::CallAwaitingResponse::HandleTimeout(ev::timer &watcher, int revents) {
+  if (remaining_timeout > 0) {
+    if (watcher.remaining() < -1.0) {
+      LOG(WARNING) << "RPC call timeout handler was delayed by "
+                   << -watcher.remaining() << "s! This may be due to a process-wide "
+                   << "pause such as swapping, logging-related delays, or allocator lock "
+                   << "contention. Will allow an additional "
+                   << remaining_timeout << "s for a response.";
+    }
+
+    watcher.set(remaining_timeout, 0);
+    watcher.start();
+    remaining_timeout = 0;
+    return;
+  }
+
+  conn->HandleOutboundCallTimeout(this);
+}
+
+void Connection::HandleOutboundCallTimeout(CallAwaitingResponse *car) {
+  DCHECK(reactor_thread_->IsCurrentThread());
+  DCHECK(car->call);
+  // The timeout timer is stopped by the car destructor exiting Connection::HandleCallResponse()
+  DCHECK(!car->call->IsFinished());
+
+  // Mark the call object as failed.
+  car->call->SetTimedOut(negotiation_complete_ ? Phase::REMOTE_CALL
+                                               : Phase::CONNECTION_NEGOTIATION);
+
+  // Test cancellation when 'car->call' is in 'TIMED_OUT' state
+  MaybeInjectCancellation(car->call);
+
+  // Drop the reference to the call. If the original caller has moved on after
+  // seeing the timeout, we no longer need to hold onto the allocated memory
+  // from the request.
+  car->call.reset();
+
+  // We still leave the CallAwaitingResponse in the map -- this is because we may still
+  // receive a response from the server, and we don't want a spurious log message
+  // when we do finally receive the response. The fact that CallAwaitingResponse::call
+  // is a NULL pointer indicates to the response processing code that the call
+  // already timed out.
+}
+
+void Connection::CancelOutboundCall(const shared_ptr<OutboundCall> &call) {
+  CallAwaitingResponse* car = FindPtrOrNull(awaiting_response_, call->call_id());
+  if (car != nullptr) {
+    // car->call may be NULL if the call has timed out already.
+    DCHECK(!car->call || car->call.get() == call.get());
+    car->call.reset();
+  }
+}
+
+// Inject a cancellation when 'call' is in state 'FLAGS_rpc_inject_cancellation_state'.
+void inline Connection::MaybeInjectCancellation(const shared_ptr<OutboundCall> &call) {
+  if (PREDICT_FALSE(call->ShouldInjectCancellation())) {
+    reactor_thread_->reactor()->messenger()->QueueCancellation(call);
+  }
+}
+
+// Callbacks after sending a call on the wire.
+// This notifies the OutboundCall object to change its state to SENT once it
+// has been fully transmitted.
+struct CallTransferCallbacks : public TransferCallbacks {
+ public:
+  explicit CallTransferCallbacks(shared_ptr<OutboundCall> call,
+                                 Connection *conn)
+      : call_(std::move(call)), conn_(conn) {}
+
+  virtual void NotifyTransferFinished() OVERRIDE {
+    // TODO: would be better to cancel the transfer while it is still on the queue if we
+    // timed out before the transfer started, but there is still a race in the case of
+    // a partial send that we have to handle here
+    if (call_->IsFinished()) {
+      DCHECK(call_->IsTimedOut() || call_->IsCancelled());
+    } else {
+      call_->SetSent();
+      // Test cancellation when 'call_' is in 'SENT' state.
+      conn_->MaybeInjectCancellation(call_);
+    }
+    delete this;
+  }
+
+  virtual void NotifyTransferAborted(const Status &status) OVERRIDE {
+    VLOG(1) << "Transfer of RPC call " << call_->ToString() << " aborted: "
+            << status.ToString();
+    delete this;
+  }
+
+ private:
+  shared_ptr<OutboundCall> call_;
+  Connection* conn_;
+};
+
+void Connection::QueueOutboundCall(shared_ptr<OutboundCall> call) {
+  DCHECK(call);
+  DCHECK_EQ(direction_, CLIENT);
+  DCHECK(reactor_thread_->IsCurrentThread());
+
+  if (PREDICT_FALSE(!shutdown_status_.ok())) {
+    // Already shutdown
+    call->SetFailed(shutdown_status_,
+                    negotiation_complete_ ? Phase::REMOTE_CALL
+                                          : Phase::CONNECTION_NEGOTIATION);
+    return;
+  }
+
+  // At this point the call has a serialized request, but no call header, since we haven't
+  // yet assigned a call ID.
+  DCHECK(!call->call_id_assigned());
+
+  // We shouldn't reach this point if 'call' was requested to be cancelled.
+  DCHECK(!call->cancellation_requested());
+
+  // Assign the call ID.
+  int32_t call_id = GetNextCallId();
+  call->set_call_id(call_id);
+
+  // Serialize the actual bytes to be put on the wire.
+  TransferPayload tmp_slices;
+  size_t n_slices = call->SerializeTo(&tmp_slices);
+
+  call->SetQueued();
+
+  // Test cancellation when 'call_' is in 'ON_OUTBOUND_QUEUE' state.
+  MaybeInjectCancellation(call);
+
+  scoped_car car(car_pool_.make_scoped_ptr(car_pool_.Construct()));
+  car->conn = this;
+  car->call = call;
+
+  // Set up the timeout timer.
+  const MonoDelta &timeout = call->controller()->timeout();
+  if (timeout.Initialized()) {
+    reactor_thread_->RegisterTimeout(&car->timeout_timer);
+    car->timeout_timer.set<CallAwaitingResponse, // NOLINT(*)
+                           &CallAwaitingResponse::HandleTimeout>(car.get());
+
+    // For calls with a timeout of at least 500ms, we actually run the timeout
+    // handler in two stages. The first timeout fires with a timeout 10% less
+    // than the user-specified one. It then schedules a second timeout for the
+    // remaining amount of time.
+    //
+    // The purpose of this two-stage timeout is to be more robust when the client
+    // has some process-wide pause, such as lock contention in tcmalloc, or a
+    // reactor callback that blocks in glog. Consider the following case:
+    //
+    // T = 0s        user issues an RPC with 5 second timeout
+    // T = 0.5s - 6s   process is blocked
+    // T = 6s        process unblocks, and the timeout fires (1s late)
+    //
+    // Without the two-stage timeout, we would determine that the call had timed out,
+    // even though it's likely that the response is waiting on our TCP socket.
+    // With the two-stage timeout, we'll end up with:
+    //
+    // T = 0s           user issues an RPC with 5 second timeout
+    // T = 0.5s - 6s    process is blocked
+    // T = 6s           process unblocks, and the first-stage timeout fires (1.5s late)
+    // T = 6s - 6.200s  time for the client to read the response which is waiting
+    // T = 6.200s       if the response was not actually available, we'll time out here
+    //
+    // We don't bother with this logic for calls with very short timeouts - assumedly
+    // a user setting such a short RPC timeout is well equipped to handle one.
+    double time = timeout.ToSeconds();
+    if (time >= 0.5) {
+      car->remaining_timeout = time * 0.1;
+      time -= car->remaining_timeout;
+    } else {
+      car->remaining_timeout = 0;
+    }
+
+    car->timeout_timer.set(time, 0);
+    car->timeout_timer.start();
+  }
+
+  TransferCallbacks *cb = new CallTransferCallbacks(std::move(call), this);
+  awaiting_response_[call_id] = car.release();
+  QueueOutbound(gscoped_ptr<OutboundTransfer>(
+      OutboundTransfer::CreateForCallRequest(call_id, tmp_slices, n_slices, cb)));
+}
+
+// Callbacks for sending an RPC call response from the server.
+// This takes ownership of the InboundCall object so that, once it has
+// been responded to, we can free up all of the associated memory.
+struct ResponseTransferCallbacks : public TransferCallbacks {
+ public:
+  ResponseTransferCallbacks(gscoped_ptr<InboundCall> call,
+                            Connection *conn) :
+    call_(std::move(call)),
+    conn_(conn)
+  {}
+
+  ~ResponseTransferCallbacks() {
+    // Remove the call from the map.
+    InboundCall *call_from_map = EraseKeyReturnValuePtr(
+      &conn_->calls_being_handled_, call_->call_id());
+    DCHECK_EQ(call_from_map, call_.get());
+  }
+
+  virtual void NotifyTransferFinished() OVERRIDE {
+    delete this;
+  }
+
+  virtual void NotifyTransferAborted(const Status &status) OVERRIDE {
+    LOG(WARNING) << "Connection torn down before " <<
+      call_->ToString() << " could send its response";
+    delete this;
+  }
+
+ private:
+  gscoped_ptr<InboundCall> call_;
+  Connection *conn_;
+};
+
+// Reactor task which puts a transfer on the outbound transfer queue.
+class QueueTransferTask : public ReactorTask {
+ public:
+  QueueTransferTask(gscoped_ptr<OutboundTransfer> transfer,
+                    Connection *conn)
+    : transfer_(std::move(transfer)),
+      conn_(conn)
+  {}
+
+  virtual void Run(ReactorThread *thr) OVERRIDE {
+    conn_->QueueOutbound(std::move(transfer_));
+    delete this;
+  }
+
+  virtual void Abort(const Status &status) OVERRIDE {
+    transfer_->Abort(status);
+    delete this;
+  }
+
+ private:
+  gscoped_ptr<OutboundTransfer> transfer_;
+  Connection *conn_;
+};
+
+void Connection::QueueResponseForCall(gscoped_ptr<InboundCall> call) {
+  // This is usually called by the IPC worker thread when the response
+  // is set, but in some circumstances may also be called by the
+  // reactor thread (e.g. if the service has shut down)
+
+  DCHECK_EQ(direction_, SERVER);
+
+  // If the connection is torn down, then the QueueOutbound() call that
+  // eventually runs in the reactor thread will take care of calling
+  // ResponseTransferCallbacks::NotifyTransferAborted.
+
+  TransferPayload tmp_slices;
+  size_t n_slices = call->SerializeResponseTo(&tmp_slices);
+
+  TransferCallbacks *cb = new ResponseTransferCallbacks(std::move(call), this);
+  // After the response is sent, can delete the InboundCall object.
+  // We set a dummy call ID and required feature set, since these are not needed
+  // when sending responses.
+  gscoped_ptr<OutboundTransfer> t(
+      OutboundTransfer::CreateForCallResponse(tmp_slices, n_slices, cb));
+
+  QueueTransferTask *task = new QueueTransferTask(std::move(t), this);
+  reactor_thread_->reactor()->ScheduleReactorTask(task);
+}
+
+void Connection::set_confidential(bool is_confidential) {
+  is_confidential_ = is_confidential;
+}
+
+bool Connection::SatisfiesCredentialsPolicy(CredentialsPolicy policy) const {
+  DCHECK_EQ(direction_, CLIENT);
+  return (policy == CredentialsPolicy::ANY_CREDENTIALS) ||
+      (policy == credentials_policy_);
+}
+
+RpczStore* Connection::rpcz_store() {
+  return reactor_thread_->reactor()->messenger()->rpcz_store();
+}
+
+void Connection::ReadHandler(ev::io &watcher, int revents) {
+  DCHECK(reactor_thread_->IsCurrentThread());
+
+  DVLOG(3) << ToString() << " ReadHandler(revents=" << revents << ")";
+  if (revents & EV_ERROR) {
+    reactor_thread_->DestroyConnection(this, Status::NetworkError(ToString() +
+                                     ": ReadHandler encountered an error"));
+    return;
+  }
+  last_activity_time_ = reactor_thread_->cur_time();
+
+  while (true) {
+    if (!inbound_) {
+      inbound_.reset(new InboundTransfer());
+    }
+    Status status = inbound_->ReceiveBuffer(*socket_);
+    if (PREDICT_FALSE(!status.ok())) {
+      if (status.posix_code() == ESHUTDOWN) {
+        VLOG(1) << ToString() << " shut down by remote end.";
+      } else {
+        LOG(WARNING) << ToString() << " recv error: " << status.ToString();
+      }
+      reactor_thread_->DestroyConnection(this, status);
+      return;
+    }
+    if (!inbound_->TransferFinished()) {
+      DVLOG(3) << ToString() << ": read is not yet finished yet.";
+      return;
+    }
+    DVLOG(3) << ToString() << ": finished reading " << inbound_->data().size() << " bytes";
+
+    if (direction_ == CLIENT) {
+      HandleCallResponse(std::move(inbound_));
+    } else if (direction_ == SERVER) {
+      HandleIncomingCall(std::move(inbound_));
+    } else {
+      LOG(FATAL) << "Invalid direction: " << direction_;
+    }
+
+    // TODO: it would seem that it would be good to loop around and see if
+    // there is more data on the socket by trying another recv(), but it turns
+    // out that it really hurts throughput to do so. A better approach
+    // might be for each InboundTransfer to actually try to read an extra byte,
+    // and if it succeeds, then we'd copy that byte into a new InboundTransfer
+    // and loop around, since it's likely the next call also arrived at the
+    // same time.
+    break;
+  }
+}
+
+void Connection::HandleIncomingCall(gscoped_ptr<InboundTransfer> transfer) {
+  DCHECK(reactor_thread_->IsCurrentThread());
+
+  gscoped_ptr<InboundCall> call(new InboundCall(this));
+  Status s = call->ParseFrom(std::move(transfer));
+  if (!s.ok()) {
+    LOG(WARNING) << ToString() << ": received bad data: " << s.ToString();
+    // TODO: shutdown? probably, since any future stuff on this socket will be
+    // "unsynchronized"
+    return;
+  }
+
+  if (!InsertIfNotPresent(&calls_being_handled_, call->call_id(), call.get())) {
+    LOG(WARNING) << ToString() << ": received call ID " << call->call_id() <<
+      " but was already processing this ID! Ignoring";
+    reactor_thread_->DestroyConnection(
+      this, Status::RuntimeError("Received duplicate call id",
+                                 Substitute("$0", call->call_id())));
+    return;
+  }
+
+  reactor_thread_->reactor()->messenger()->QueueInboundCall(std::move(call));
+}
+
+void Connection::HandleCallResponse(gscoped_ptr<InboundTransfer> transfer) {
+  DCHECK(reactor_thread_->IsCurrentThread());
+  gscoped_ptr<CallResponse> resp(new CallResponse);
+  CHECK_OK(resp->ParseFrom(std::move(transfer)));
+
+  CallAwaitingResponse *car_ptr =
+    EraseKeyReturnValuePtr(&awaiting_response_, resp->call_id());
+  if (PREDICT_FALSE(car_ptr == nullptr)) {
+    LOG(WARNING) << ToString() << ": Got a response for call id " << resp->call_id() << " which "
+                 << "was not pending! Ignoring.";
+    return;
+  }
+
+  // The car->timeout_timer ev::timer will be stopped automatically by its destructor.
+  scoped_car car(car_pool_.make_scoped_ptr(car_ptr));
+
+  if (PREDICT_FALSE(!car->call)) {
+    // The call already failed due to a timeout.
+    VLOG(1) << "Got response to call id " << resp->call_id() << " after client "
+            << "already timed out or cancelled";
+    return;
+  }
+
+  car->call->SetResponse(std::move(resp));
+
+  // Test cancellation when 'car->call' is in 'FINISHED_SUCCESS' or 'FINISHED_ERROR' state.
+  MaybeInjectCancellation(car->call);
+}
+
+void Connection::WriteHandler(ev::io &watcher, int revents) {
+  DCHECK(reactor_thread_->IsCurrentThread());
+
+  if (revents & EV_ERROR) {
+    reactor_thread_->DestroyConnection(this, Status::NetworkError(ToString() +
+          ": writeHandler encountered an error"));
+    return;
+  }
+  DVLOG(3) << ToString() << ": writeHandler: revents = " << revents;
+
+  OutboundTransfer *transfer;
+  if (outbound_transfers_.empty()) {
+    LOG(WARNING) << ToString() << " got a ready-to-write callback, but there is "
+      "nothing to write.";
+    write_io_.stop();
+    return;
+  }
+
+  while (!outbound_transfers_.empty()) {
+    transfer = &(outbound_transfers_.front());
+
+    if (!transfer->TransferStarted()) {
+
+      if (transfer->is_for_outbound_call()) {
+        CallAwaitingResponse* car = FindOrDie(awaiting_response_, transfer->call_id());
+        if (!car->call) {
+          // If the call has already timed out or has already been cancelled, the 'call'
+          // field would be set to NULL. In that case, don't bother sending it.
+          outbound_transfers_.pop_front();
+          transfer->Abort(Status::Aborted("already timed out or cancelled"));
+          delete transfer;
+          continue;
+        }
+
+        // If this is the start of the transfer, then check if the server has the
+        // required RPC flags. We have to wait until just before the transfer in
+        // order to ensure that the negotiation has taken place, so that the flags
+        // are available.
+        const set<RpcFeatureFlag>& required_features = car->call->required_rpc_features();
+        if (!includes(remote_features_.begin(), remote_features_.end(),
+                      required_features.begin(), required_features.end())) {
+          outbound_transfers_.pop_front();
+          Status s = Status::NotSupported("server does not support the required RPC features");
+          transfer->Abort(s);
+          Phase phase = negotiation_complete_ ? Phase::REMOTE_CALL : Phase::CONNECTION_NEGOTIATION;
+          car->call->SetFailed(std::move(s), phase);
+          // Test cancellation when 'call_' is in 'FINISHED_ERROR' state.
+          MaybeInjectCancellation(car->call);
+          car->call.reset();
+          delete transfer;
+          continue;
+        }
+
+        car->call->SetSending();
+
+        // Test cancellation when 'call_' is in 'SENDING' state.
+        MaybeInjectCancellation(car->call);
+      }
+    }
+
+    last_activity_time_ = reactor_thread_->cur_time();
+    Status status = transfer->SendBuffer(*socket_);
+    if (PREDICT_FALSE(!status.ok())) {
+      LOG(WARNING) << ToString() << " send error: " << status.ToString();
+      reactor_thread_->DestroyConnection(this, status);
+      return;
+    }
+
+    if (!transfer->TransferFinished()) {
+      DVLOG(3) << ToString() << ": writeHandler: xfer not finished.";
+      return;
+    }
+
+    outbound_transfers_.pop_front();
+    delete transfer;
+  }
+
+  // If we were able to write all of our outbound transfers,
+  // we don't have any more to write.
+  write_io_.stop();
+}
+
+std::string Connection::ToString() const {
+  // This may be called from other threads, so we cannot
+  // include anything in the output about the current state,
+  // which might concurrently change from another thread.
+  return strings::Substitute(
+    "$0 $1",
+    direction_ == SERVER ? "server connection from" : "client connection to",
+    remote_.ToString());
+}
+
+// Reactor task that transitions this Connection from connection negotiation to
+// regular RPC handling. Destroys Connection on negotiation error.
+class NegotiationCompletedTask : public ReactorTask {
+ public:
+  NegotiationCompletedTask(Connection* conn,
+                           Status negotiation_status,
+                           std::unique_ptr<ErrorStatusPB> rpc_error)
+    : conn_(conn),
+      negotiation_status_(std::move(negotiation_status)),
+      rpc_error_(std::move(rpc_error)) {
+  }
+
+  virtual void Run(ReactorThread *rthread) OVERRIDE {
+    rthread->CompleteConnectionNegotiation(conn_,
+                                           negotiation_status_,
+                                           std::move(rpc_error_));
+    delete this;
+  }
+
+  virtual void Abort(const Status &status) OVERRIDE {
+    DCHECK(conn_->reactor_thread()->reactor()->closing());
+    VLOG(1) << "Failed connection negotiation due to shut down reactor thread: "
+            << status.ToString();
+    delete this;
+  }
+
+ private:
+  scoped_refptr<Connection> conn_;
+  const Status negotiation_status_;
+  std::unique_ptr<ErrorStatusPB> rpc_error_;
+};
+
+void Connection::CompleteNegotiation(Status negotiation_status,
+                                     unique_ptr<ErrorStatusPB> rpc_error) {
+  auto task = new NegotiationCompletedTask(
+      this, std::move(negotiation_status), std::move(rpc_error));
+  reactor_thread_->reactor()->ScheduleReactorTask(task);
+}
+
+void Connection::MarkNegotiationComplete() {
+  DCHECK(reactor_thread_->IsCurrentThread());
+  negotiation_complete_ = true;
+}
+
+Status Connection::DumpPB(const DumpRunningRpcsRequestPB& req,
+                          RpcConnectionPB* resp) {
+  DCHECK(reactor_thread_->IsCurrentThread());
+  resp->set_remote_ip(remote_.ToString());
+  if (negotiation_complete_) {
+    resp->set_state(RpcConnectionPB::OPEN);
+  } else {
+    resp->set_state(RpcConnectionPB::NEGOTIATING);
+  }
+
+  if (direction_ == CLIENT) {
+    for (const car_map_t::value_type& entry : awaiting_response_) {
+      CallAwaitingResponse *c = entry.second;
+      if (c->call) {
+        c->call->DumpPB(req, resp->add_calls_in_flight());
+      }
+    }
+
+    resp->set_outbound_queue_size(num_queued_outbound_transfers());
+  } else if (direction_ == SERVER) {
+    if (negotiation_complete_) {
+      // It's racy to dump credentials while negotiating, since the Connection
+      // object is owned by the negotiation thread at that point.
+      resp->set_remote_user_credentials(remote_user_.ToString());
+    }
+    for (const inbound_call_map_t::value_type& entry : calls_being_handled_) {
+      InboundCall* c = entry.second;
+      c->DumpPB(req, resp->add_calls_in_flight());
+    }
+  } else {
+    LOG(FATAL);
+  }
+  return Status::OK();
+}
+
+} // namespace rpc
+} // namespace kudu
diff --git a/be/src/kudu/rpc/connection.h b/be/src/kudu/rpc/connection.h
new file mode 100644
index 0000000..362a35b
--- /dev/null
+++ b/be/src/kudu/rpc/connection.h
@@ -0,0 +1,391 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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.
+
+#ifndef KUDU_RPC_CONNECTION_H
+#define KUDU_RPC_CONNECTION_H
+
+#include <cstddef>
+#include <cstdint>
+#include <limits>
+#include <memory>
+#include <set>
+#include <string>
+#include <unordered_map>
+#include <utility>
+
+#include <boost/intrusive/list.hpp>
+#include <boost/optional/optional.hpp>
+#include <ev++.h>
+#include <glog/logging.h>
+
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/rpc/connection_id.h"
+#include "kudu/rpc/rpc_controller.h"
+#include "kudu/rpc/rpc_header.pb.h"
+#include "kudu/rpc/remote_user.h"
+#include "kudu/rpc/transfer.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/net/socket.h"
+#include "kudu/util/object_pool.h"
+#include "kudu/util/status.h"
+
+namespace kudu {
+
+namespace rpc {
+
+class DumpRunningRpcsRequestPB;
+class InboundCall;
+class OutboundCall;
+class RpcConnectionPB;
+class ReactorThread;
+class RpczStore;
+enum class CredentialsPolicy;
+
+//
+// A connection between an endpoint and us.
+//
+// Inbound connections are created by AcceptorPools, which eventually schedule
+// RegisterConnection() to be called from the reactor thread.
+//
+// Outbound connections are created by the Reactor thread in order to service
+// outbound calls.
+//
+// Once a Connection is created, it can be used both for sending messages and
+// receiving them, but any given connection is explicitly a client or server.
+// If a pair of servers are making bidirectional RPCs, they will use two separate
+// TCP connections (and Connection objects).
+//
+// This class is not fully thread-safe.  It is accessed only from the context of a
+// single ReactorThread except where otherwise specified.
+//
+class Connection : public RefCountedThreadSafe<Connection> {
+ public:
+  enum Direction {
+    // This host is sending calls via this connection.
+    CLIENT,
+    // This host is receiving calls via this connection.
+    SERVER
+  };
+
+  // Create a new Connection.
+  // reactor_thread: the reactor that owns us.
+  // remote: the address of the remote end
+  // socket: the socket to take ownership of.
+  // direction: whether we are the client or server side
+  Connection(ReactorThread *reactor_thread,
+             Sockaddr remote,
+             std::unique_ptr<Socket> socket,
+             Direction direction,
+             CredentialsPolicy policy = CredentialsPolicy::ANY_CREDENTIALS);
+
+  // Set underlying socket to non-blocking (or blocking) mode.
+  Status SetNonBlocking(bool enabled);
+
+  // Register our socket with an epoll loop.  We will only ever be registered in
+  // one epoll loop at a time.
+  void EpollRegister(ev::loop_ref& loop);
+
+  ~Connection();
+
+  MonoTime last_activity_time() const {
+    return last_activity_time_;
+  }
+
+  // Returns true if we are not in the process of receiving or sending a
+  // message, and we have no outstanding calls.
+  bool Idle() const;
+
+  // Fail any calls which are currently queued or awaiting response.
+  // Prohibits any future calls (they will be failed immediately with this
+  // same Status).
+  void Shutdown(const Status& status,
+                std::unique_ptr<ErrorStatusPB> rpc_error = {});
+
+  // Queue a new call to be made. If the queueing fails, the call will be
+  // marked failed. The caller is expected to check if 'call' has been cancelled
+  // before making the call.
+  // Takes ownership of the 'call' object regardless of whether it succeeds or fails.
+  void QueueOutboundCall(std::shared_ptr<OutboundCall> call);
+
+  // Queue a call response back to the client on the server side.
+  //
+  // This may be called from a non-reactor thread.
+  void QueueResponseForCall(gscoped_ptr<InboundCall> call);
+
+  // Cancel an outbound call by removing any reference to it by CallAwaitingResponse
+  // in 'awaiting_responses_'.
+  void CancelOutboundCall(const std::shared_ptr<OutboundCall> &call);
+
+  // The address of the remote end of the connection.
+  const Sockaddr &remote() const { return remote_; }
+
+  // Set the user credentials for an outbound connection.
+  void set_outbound_connection_id(ConnectionId conn_id) {
+    DCHECK_EQ(direction_, CLIENT);
+    DCHECK(!outbound_connection_id_);
+    outbound_connection_id_ = std::move(conn_id);
+  }
+
+  // Get the user credentials which will be used to log in.
+  const ConnectionId& outbound_connection_id() const {
+    DCHECK_EQ(direction_, CLIENT);
+    DCHECK(outbound_connection_id_);
+    return *outbound_connection_id_;
+  }
+
+  bool is_confidential() const {
+    return is_confidential_;
+  }
+
+  // Set/unset the 'confidentiality' property for this connection.
+  void set_confidential(bool is_confidential);
+
+  // Credentials policy to start connection negotiation.
+  CredentialsPolicy credentials_policy() const { return credentials_policy_; }
+
+  // Whether the connection satisfies the specified credentials policy.
+  //
+  // NOTE: The policy is set prior to connection negotiation, and the actual
+  //       authentication credentials used for connection negotiation might
+  //       effectively make the connection to satisfy a stronger policy.
+  //       An example: the credentials policy for the connection was set to
+  //       ANY_CREDENTIALS, but since the authn token was not available
+  //       at the time of negotiation, the primary credentials were used, making
+  //       the connection de facto satisfying the PRIMARY_CREDENTIALS policy.
+  bool SatisfiesCredentialsPolicy(CredentialsPolicy policy) const;
+
+  RpczStore* rpcz_store();
+
+  // libev callback when data is available to read.
+  void ReadHandler(ev::io &watcher, int revents);
+
+  // libev callback when we may write to the socket.
+  void WriteHandler(ev::io &watcher, int revents);
+
+  // Safe to be called from other threads.
+  std::string ToString() const;
+
+  Direction direction() const { return direction_; }
+
+  Socket* socket() { return socket_.get(); }
+
+  // Go through the process of transferring control of the underlying socket back to the Reactor.
+  void CompleteNegotiation(Status negotiation_status,
+                           std::unique_ptr<ErrorStatusPB> rpc_error);
+
+  // Indicate that negotiation is complete and that the Reactor is now in control of the socket.
+  void MarkNegotiationComplete();
+
+  Status DumpPB(const DumpRunningRpcsRequestPB& req,
+                RpcConnectionPB* resp);
+
+  ReactorThread* reactor_thread() const { return reactor_thread_; }
+
+  std::unique_ptr<Socket> release_socket() {
+    return std::move(socket_);
+  }
+
+  void adopt_socket(std::unique_ptr<Socket> socket) {
+    socket_ = std::move(socket);
+  }
+
+  void set_remote_features(std::set<RpcFeatureFlag> remote_features) {
+    remote_features_ = std::move(remote_features);
+  }
+
+  void set_remote_user(RemoteUser user) {
+    DCHECK_EQ(direction_, SERVER);
+    remote_user_ = std::move(user);
+  }
+
+  const RemoteUser& remote_user() const {
+    DCHECK_EQ(direction_, SERVER);
+    return remote_user_;
+  }
+
+  // Whether the connection is scheduled for shutdown.
+  bool scheduled_for_shutdown() const {
+    DCHECK_EQ(direction_, CLIENT);
+    return scheduled_for_shutdown_;
+  }
+
+  // Mark the connection as scheduled to be shut down. Reactor does not dispatch
+  // new calls on such a connection.
+  void set_scheduled_for_shutdown() {
+    DCHECK_EQ(direction_, CLIENT);
+    scheduled_for_shutdown_ = true;
+  }
+
+  size_t num_queued_outbound_transfers() const {
+    return outbound_transfers_.size();
+  }
+
+ private:
+  friend struct CallAwaitingResponse;
+  friend class QueueTransferTask;
+  friend struct CallTransferCallbacks;
+  friend struct ResponseTransferCallbacks;
+
+  // A call which has been fully sent to the server, which we're waiting for
+  // the server to process. This is used on the client side only.
+  struct CallAwaitingResponse {
+    ~CallAwaitingResponse();
+
+    // Notification from libev that the call has timed out.
+    void HandleTimeout(ev::timer &watcher, int revents);
+
+    Connection *conn;
+    std::shared_ptr<OutboundCall> call;
+    ev::timer timeout_timer;
+
+    // We time out RPC calls in two stages. This is set to the amount of timeout
+    // remaining after the next timeout fires. See Connection::QueueOutboundCall().
+    double remaining_timeout;
+  };
+
+  typedef std::unordered_map<uint64_t, CallAwaitingResponse*> car_map_t;
+  typedef std::unordered_map<uint64_t, InboundCall*> inbound_call_map_t;
+
+  // Returns the next valid (positive) sequential call ID by incrementing a counter
+  // and ensuring we roll over from INT32_MAX to 0.
+  // Negative numbers are reserved for special purposes.
+  int32_t GetNextCallId() {
+    int32_t call_id = next_call_id_;
+    if (PREDICT_FALSE(next_call_id_ == std::numeric_limits<int32_t>::max())) {
+      next_call_id_ = 0;
+    } else {
+      next_call_id_++;
+    }
+    return call_id;
+  }
+
+  // An incoming packet has completed transferring on the server side.
+  // This parses the call and delivers it into the call queue.
+  void HandleIncomingCall(gscoped_ptr<InboundTransfer> transfer);
+
+  // An incoming packet has completed on the client side. This parses the
+  // call response, looks up the CallAwaitingResponse, and calls the
+  // client callback.
+  void HandleCallResponse(gscoped_ptr<InboundTransfer> transfer);
+
+  // The given CallAwaitingResponse has elapsed its user-defined timeout.
+  // Set it to Failed.
+  void HandleOutboundCallTimeout(CallAwaitingResponse *car);
+
+  // Queue a transfer for sending on this connection.
+  // We will take ownership of the transfer.
+  // This must be called from the reactor thread.
+  void QueueOutbound(gscoped_ptr<OutboundTransfer> transfer);
+
+  // Internal test function for injecting cancellation request when 'call'
+  // reaches state specified in 'FLAGS_rpc_inject_cancellation_state'.
+  void MaybeInjectCancellation(const std::shared_ptr<OutboundCall> &call);
+
+  // The reactor thread that created this connection.
+  ReactorThread* const reactor_thread_;
+
+  // The remote address we're talking to.
+  const Sockaddr remote_;
+
+  // The socket we're communicating on.
+  std::unique_ptr<Socket> socket_;
+
+  // The ConnectionId that serves as a key into the client connection map
+  // within this reactor. Only set in the case of outbound connections.
+  boost::optional<ConnectionId> outbound_connection_id_;
+
+  // The authenticated remote user (if this is an inbound connection on the server).
+  RemoteUser remote_user_;
+
+  // whether we are client or server
+  Direction direction_;
+
+  // The last time we read or wrote from the socket.
+  MonoTime last_activity_time_;
+
+  // the inbound transfer, if any
+  gscoped_ptr<InboundTransfer> inbound_;
+
+  // notifies us when our socket is writable.
+  ev::io write_io_;
+
+  // notifies us when our socket is readable.
+  ev::io read_io_;
+
+  // Set to true when the connection is registered on a loop.
+  // This is used for a sanity check in the destructor that we are properly
+  // un-registered before shutting down.
+  bool is_epoll_registered_;
+
+  // waiting to be sent
+  boost::intrusive::list<OutboundTransfer> outbound_transfers_; // NOLINT(*)
+
+  // Calls which have been sent and are now waiting for a response.
+  car_map_t awaiting_response_;
+
+  // Calls which have been received on the server and are currently
+  // being handled.
+  inbound_call_map_t calls_being_handled_;
+
+  // the next call ID to use
+  int32_t next_call_id_;
+
+  // Starts as Status::OK, gets set to a shutdown status upon Shutdown().
+  Status shutdown_status_;
+
+  // RPC features supported by the remote end of the connection.
+  std::set<RpcFeatureFlag> remote_features_;
+
+  // Pool from which CallAwaitingResponse objects are allocated.
+  // Also a funny name.
+  ObjectPool<CallAwaitingResponse> car_pool_;
+  typedef ObjectPool<CallAwaitingResponse>::scoped_ptr scoped_car;
+
+  // The credentials policy to use for connection negotiation. It defines which
+  // type of user credentials used to negotiate a connection. The actual type of
+  // credentials used for authentication during the negotiation process depends
+  // on the credentials availability, but the result credentials guaranteed to
+  // always satisfy the specified credentials policy. In other words, the actual
+  // type of credentials used for connection negotiation might effectively make
+  // the connection to satisfy a stronger/narrower policy.
+  //
+  // An example:
+  //   The credentials policy for the connection was set to ANY_CREDENTIALS,
+  //   but since no secondary credentials (such authn token) were available
+  //   at the time of negotiation, the primary credentials were used,making the
+  //   connection satisfying the PRIMARY_CREDENTIALS policy de facto.
+  const CredentialsPolicy credentials_policy_;
+
+  // Whether we completed connection negotiation.
+  bool negotiation_complete_;
+
+  // Whether it's OK to pass confidential information over the connection.
+  // For example, an encrypted (but not necessarily authenticated) connection
+  // is considered confidential.
+  bool is_confidential_;
+
+  // Whether the connection is scheduled for shutdown.
+  bool scheduled_for_shutdown_;
+};
+
+} // namespace rpc
+} // namespace kudu
+
+#endif
diff --git a/be/src/kudu/rpc/connection_id.cc b/be/src/kudu/rpc/connection_id.cc
new file mode 100644
index 0000000..6720807
--- /dev/null
+++ b/be/src/kudu/rpc/connection_id.cc
@@ -0,0 +1,85 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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.
+
+#include "kudu/rpc/connection_id.h"
+
+#include <cstddef>
+#include <utility>
+
+#include <boost/functional/hash/hash.hpp>
+#include <glog/logging.h>
+
+#include "kudu/gutil/strings/substitute.h"
+
+using std::string;
+
+namespace kudu {
+namespace rpc {
+
+ConnectionId::ConnectionId() {}
+
+ConnectionId::ConnectionId(const Sockaddr& remote,
+                           std::string hostname,
+                           UserCredentials user_credentials)
+    : remote_(remote),
+      hostname_(std::move(hostname)),
+      user_credentials_(std::move(user_credentials)) {
+  CHECK(!hostname_.empty());
+}
+
+void ConnectionId::set_user_credentials(UserCredentials user_credentials) {
+  DCHECK(user_credentials.has_real_user());
+  user_credentials_ = std::move(user_credentials);
+}
+
+string ConnectionId::ToString() const {
+  string remote;
+  if (hostname_ != remote_.host()) {
+    remote = strings::Substitute("$0 ($1)", remote_.ToString(), hostname_);
+  } else {
+    remote = remote_.ToString();
+  }
+
+  return strings::Substitute("{remote=$0, user_credentials=$1}",
+                             remote,
+                             user_credentials_.ToString());
+}
+
+size_t ConnectionId::HashCode() const {
+  size_t seed = 0;
+  boost::hash_combine(seed, remote_.HashCode());
+  boost::hash_combine(seed, hostname_);
+  boost::hash_combine(seed, user_credentials_.HashCode());
+  return seed;
+}
+
+bool ConnectionId::Equals(const ConnectionId& other) const {
+  return remote() == other.remote() &&
+      hostname_ == other.hostname_ &&
+      user_credentials().Equals(other.user_credentials());
+}
+
+size_t ConnectionIdHash::operator() (const ConnectionId& conn_id) const {
+  return conn_id.HashCode();
+}
+
+bool ConnectionIdEqual::operator() (const ConnectionId& cid1, const ConnectionId& cid2) const {
+  return cid1.Equals(cid2);
+}
+
+} // namespace rpc
+} // namespace kudu
diff --git a/be/src/kudu/rpc/connection_id.h b/be/src/kudu/rpc/connection_id.h
new file mode 100644
index 0000000..67a4786
--- /dev/null
+++ b/be/src/kudu/rpc/connection_id.h
@@ -0,0 +1,84 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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.
+#pragma once
+
+#include <cstddef>
+#include <string>
+
+#include "kudu/rpc/user_credentials.h"
+#include "kudu/util/net/sockaddr.h"
+
+namespace kudu {
+namespace rpc {
+
+// Used to key on Connection information.
+// For use as a key in an unordered STL collection, use ConnectionIdHash and ConnectionIdEqual.
+// This class is copyable for STL compatibility, but not assignable (use CopyFrom() for that).
+class ConnectionId {
+ public:
+  ConnectionId();
+
+  // Copy constructor required for use with STL unordered_map.
+  ConnectionId(const ConnectionId& other) = default;
+
+  // Convenience constructor.
+  ConnectionId(const Sockaddr& remote,
+               std::string hostname,
+               UserCredentials user_credentials);
+
+  // The remote address.
+  const Sockaddr& remote() const { return remote_; }
+
+  const std::string& hostname() const { return hostname_; }
+
+  // The credentials of the user associated with this connection, if any.
+  void set_user_credentials(UserCredentials user_credentials);
+
+  const UserCredentials& user_credentials() const { return user_credentials_; }
+
+  // Copy state from another object to this one.
+  void CopyFrom(const ConnectionId& other);
+
+  // Returns a string representation of the object, not including the password field.
+  std::string ToString() const;
+
+  size_t HashCode() const;
+  bool Equals(const ConnectionId& other) const;
+
+ private:
+  // Remember to update HashCode() and Equals() when new fields are added.
+  Sockaddr remote_;
+
+  // The original host name before it was resolved to 'remote_'.
+  // This must be retained since it is used to compute Kerberos Service Principal Names (SPNs).
+  std::string hostname_;
+
+  UserCredentials user_credentials_;
+};
+
+class ConnectionIdHash {
+ public:
+  std::size_t operator() (const ConnectionId& conn_id) const;
+};
+
+class ConnectionIdEqual {
+ public:
+  bool operator() (const ConnectionId& cid1, const ConnectionId& cid2) const;
+};
+
+} // namespace rpc
+} // namespace kudu
diff --git a/be/src/kudu/rpc/constants.cc b/be/src/kudu/rpc/constants.cc
new file mode 100644
index 0000000..a4e024c
--- /dev/null
+++ b/be/src/kudu/rpc/constants.cc
@@ -0,0 +1,37 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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.
+
+#include "kudu/rpc/constants.h"
+
+using std::set;
+
+namespace kudu {
+namespace rpc {
+
+const char* const kMagicNumber = "hrpc";
+const char* const kSaslAppName = "kudu";
+
+// NOTE: the TLS flag is dynamically added based on the local encryption
+// configuration.
+//
+// NOTE: the TLS_AUTHENTICATION_ONLY flag is dynamically added on both
+// sides based on the remote peer's address.
+set<RpcFeatureFlag> kSupportedServerRpcFeatureFlags = { APPLICATION_FEATURE_FLAGS };
+set<RpcFeatureFlag> kSupportedClientRpcFeatureFlags = { APPLICATION_FEATURE_FLAGS };
+
+} // namespace rpc
+} // namespace kudu
diff --git a/be/src/kudu/rpc/constants.h b/be/src/kudu/rpc/constants.h
new file mode 100644
index 0000000..a3c7c67
--- /dev/null
+++ b/be/src/kudu/rpc/constants.h
@@ -0,0 +1,60 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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.
+
+#ifndef KUDU_RPC_RPC_CONSTANTS_H
+#define KUDU_RPC_RPC_CONSTANTS_H
+
+#include <cstdint>
+#include <set>
+
+#include "kudu/rpc/rpc_header.pb.h"
+
+namespace kudu {
+namespace rpc {
+
+// Magic number bytes sent at connection setup time.
+extern const char* const kMagicNumber;
+
+// App name for SASL library init
+extern const char* const kSaslAppName;
+
+// Current version of the RPC protocol.
+static const uint32_t kCurrentRpcVersion = 9;
+
+// From Hadoop.
+static const int32_t kInvalidCallId = -2;
+static const int32_t kConnectionContextCallId = -3;
+static const int32_t kNegotiateCallId = -33;
+
+static const uint8_t kMagicNumberLength = 4;
+static const uint8_t kHeaderFlagsLength = 3;
+
+// There is a 4-byte length prefix before any packet.
+static const uint8_t kMsgLengthPrefixLength = 4;
+
+// The set of RPC features that this server build supports.
+// Non-const for testing.
+extern std::set<RpcFeatureFlag> kSupportedServerRpcFeatureFlags;
+
+// The set of RPC features that this client build supports.
+// Non-const for testing.
+extern std::set<RpcFeatureFlag> kSupportedClientRpcFeatureFlags;
+
+} // namespace rpc
+} // namespace kudu
+
+#endif // KUDU_RPC_RPC_CONSTANTS_H
diff --git a/be/src/kudu/rpc/exactly_once_rpc-test.cc b/be/src/kudu/rpc/exactly_once_rpc-test.cc
new file mode 100644
index 0000000..c94e89c
--- /dev/null
+++ b/be/src/kudu/rpc/exactly_once_rpc-test.cc
@@ -0,0 +1,629 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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.
+
+#include <unistd.h>
+
+#include <atomic>
+#include <cstdint>
+#include <cstdlib>
+#include <memory>
+#include <ostream>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/callback.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/rpc/request_tracker.h"
+#include "kudu/rpc/response_callback.h"
+#include "kudu/rpc/result_tracker.h"
+#include "kudu/rpc/retriable_rpc.h"
+#include "kudu/rpc/rpc-test-base.h"
+#include "kudu/rpc/rpc.h"
+#include "kudu/rpc/rpc_controller.h"
+#include "kudu/rpc/rpc_header.pb.h"
+#include "kudu/rpc/rtest.pb.h"
+#include "kudu/rpc/rtest.proxy.h"
+#include "kudu/util/countdown_latch.h"
+#include "kudu/util/mem_tracker.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/pb_util.h"
+#include "kudu/util/status.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+#include "kudu/util/thread.h"
+
+DECLARE_int64(remember_clients_ttl_ms);
+DECLARE_int64(remember_responses_ttl_ms);
+DECLARE_int64(result_tracker_gc_interval_ms);
+
+using kudu::pb_util::SecureDebugString;
+using kudu::pb_util::SecureShortDebugString;
+using std::atomic_int;
+using std::shared_ptr;
+using std::unique_ptr;
+using std::vector;
+
+namespace kudu {
+namespace rpc {
+
+class Messenger;
+
+namespace {
+
+const char* kClientId = "test-client";
+
+void AddRequestId(RpcController* controller,
+                  const std::string& client_id,
+                  ResultTracker::SequenceNumber sequence_number,
+                  int64_t attempt_no) {
+  unique_ptr<RequestIdPB> request_id(new RequestIdPB());
+  request_id->set_client_id(client_id);
+  request_id->set_seq_no(sequence_number);
+  request_id->set_attempt_no(attempt_no);
+  request_id->set_first_incomplete_seq_no(sequence_number);
+  controller->SetRequestIdPB(std::move(request_id));
+}
+
+class TestServerPicker : public ServerPicker<CalculatorServiceProxy> {
+ public:
+  explicit TestServerPicker(CalculatorServiceProxy* proxy) : proxy_(proxy) {}
+
+  void PickLeader(const ServerPickedCallback& callback, const MonoTime& deadline) override {
+    callback.Run(Status::OK(), proxy_);
+  }
+
+  void MarkServerFailed(CalculatorServiceProxy*, const Status&) override {}
+  void MarkReplicaNotLeader(CalculatorServiceProxy*) override {}
+  void MarkResourceNotFound(CalculatorServiceProxy*) override {}
+
+ private:
+  CalculatorServiceProxy* proxy_;
+};
+
+} // anonymous namespace
+
+class CalculatorServiceRpc : public RetriableRpc<CalculatorServiceProxy,
+                                                 ExactlyOnceRequestPB,
+                                                 ExactlyOnceResponsePB> {
+ public:
+  CalculatorServiceRpc(const scoped_refptr<TestServerPicker>& server_picker,
+                       const scoped_refptr<RequestTracker>& request_tracker,
+                       const MonoTime& deadline,
+                       shared_ptr<Messenger> messenger,
+                       int value,
+                       CountDownLatch* latch,
+                       int server_sleep = 0)
+      : RetriableRpc(server_picker, request_tracker, deadline, std::move(messenger)),
+        latch_(latch) {
+    req_.set_value_to_add(value);
+    req_.set_randomly_fail(true);
+    req_.set_sleep_for_ms(server_sleep);
+  }
+
+  void Try(CalculatorServiceProxy* server, const ResponseCallback& callback) override {
+    server->AddExactlyOnceAsync(req_,
+                                &resp_,
+                                mutable_retrier()->mutable_controller(),
+                                callback);
+  }
+
+  RetriableRpcStatus AnalyzeResponse(const Status& rpc_cb_status) override {
+    // We shouldn't get errors from the server/rpc system since we set a high timeout.
+    CHECK_OK(rpc_cb_status);
+
+    if (!mutable_retrier()->controller().status().ok()) {
+      CHECK(mutable_retrier()->controller().status().IsRemoteError());
+      if (mutable_retrier()->controller().error_response()->code()
+          == ErrorStatusPB::ERROR_REQUEST_STALE) {
+        return { RetriableRpcStatus::NON_RETRIABLE_ERROR,
+              mutable_retrier()->controller().status() };
+      }
+      return { RetriableRpcStatus::SERVICE_UNAVAILABLE,
+               mutable_retrier()->controller().status() };
+    }
+
+    // If the controller is not finished we're in the ReplicaFoundCb() callback.
+    // Return ok to proceed with the call to the server.
+    if (!mutable_retrier()->mutable_controller()->finished()) {
+      return { RetriableRpcStatus::OK, Status::OK() };
+    }
+
+    // If we've received a response in the past, all following responses must
+    // match.
+    if (!successful_response_.IsInitialized()) {
+      successful_response_.CopyFrom(resp_);
+    } else {
+      CHECK_EQ(SecureDebugString(successful_response_),
+               SecureDebugString(resp_));
+    }
+
+    if (sometimes_retry_successful_) {
+      // Still report errors, with some probability. This will cause requests to
+      // be retried. Since the requests were originally successful we should get
+      // the same reply back.
+      int random = rand() % 4;
+      switch (random) {
+        case 0:
+          return { RetriableRpcStatus::SERVICE_UNAVAILABLE,
+                   Status::RemoteError("") };
+        case 1:
+          return { RetriableRpcStatus::RESOURCE_NOT_FOUND,
+                   Status::RemoteError("") };
+        case 2:
+          return { RetriableRpcStatus::SERVER_NOT_ACCESSIBLE,
+                   Status::RemoteError("") };
+        case 3:
+          return { RetriableRpcStatus::OK, Status::OK() };
+        default: LOG(FATAL) << "Unexpected value";
+      }
+    }
+    return { RetriableRpcStatus::OK, Status::OK() };
+  }
+
+  void Finish(const Status& status) override {
+    CHECK_OK(status);
+    latch_->CountDown();
+    delete this;
+  }
+
+  std::string ToString() const override { return "test-rpc"; }
+  CountDownLatch* latch_;
+  ExactlyOnceResponsePB successful_response_;
+  bool sometimes_retry_successful_ = true;
+};
+
+class ExactlyOnceRpcTest : public RpcTestBase {
+ public:
+  void SetUp() override {
+    RpcTestBase::SetUp();
+    SeedRandom();
+  }
+
+  Status StartServer() {
+    // Set up server.
+    RETURN_NOT_OK(StartTestServerWithGeneratedCode(&server_addr_));
+    RETURN_NOT_OK(CreateMessenger("Client", &client_messenger_));
+    proxy_.reset(new CalculatorServiceProxy(
+        client_messenger_, server_addr_, server_addr_.host()));
+    test_picker_.reset(new TestServerPicker(proxy_.get()));
+    request_tracker_.reset(new RequestTracker(kClientId));
+    attempt_nos_ = 0;
+
+    return Status::OK();
+  }
+
+  // An exactly once adder that uses RetriableRpc to perform the requests.
+  struct RetriableRpcExactlyOnceAdder {
+    RetriableRpcExactlyOnceAdder(const scoped_refptr<TestServerPicker>& server_picker,
+                     const scoped_refptr<RequestTracker>& request_tracker,
+                     shared_ptr<Messenger> messenger,
+                     int value,
+                     int server_sleep = 0) : latch_(1) {
+      MonoTime now = MonoTime::Now();
+      now.AddDelta(MonoDelta::FromMilliseconds(10000));
+      rpc_ = new CalculatorServiceRpc(server_picker,
+                                      request_tracker,
+                                      now,
+                                      std::move(messenger),
+                                      value,
+                                      &latch_,
+                                      server_sleep);
+    }
+
+    void Start() {
+      CHECK_OK(kudu::Thread::Create(
+                   "test",
+                   "test",
+                   &RetriableRpcExactlyOnceAdder::SleepAndSend, this, &thread));
+    }
+
+    void SleepAndSend() {
+      rpc_->SendRpc();
+      latch_.Wait();
+    }
+
+    CountDownLatch latch_;
+    scoped_refptr<kudu::Thread> thread;
+    CalculatorServiceRpc* rpc_;
+  };
+
+  // An exactly once adder that sends multiple, simultaneous calls, to the server
+  // and makes sure that only one of the calls was successful.
+  struct SimultaneousExactlyOnceAdder {
+    SimultaneousExactlyOnceAdder(CalculatorServiceProxy* p,
+                     ResultTracker::SequenceNumber sequence_number,
+                     int value,
+                     uint64_t client_sleep,
+                     uint64_t server_sleep,
+                     int64_t attempt_no)
+     : proxy(p),
+       client_sleep_for_ms(client_sleep) {
+      req.set_value_to_add(value);
+      req.set_sleep_for_ms(server_sleep);
+      AddRequestId(&controller, kClientId, sequence_number, attempt_no);
+    }
+
+    void Start() {
+      CHECK_OK(kudu::Thread::Create(
+          "test",
+          "test",
+          &SimultaneousExactlyOnceAdder::SleepAndSend, this, &thread));
+    }
+
+    // Sleeps the preset number of msecs before sending the call.
+    void SleepAndSend() {
+      usleep(client_sleep_for_ms * 1000);
+      controller.set_timeout(MonoDelta::FromSeconds(20));
+      CHECK_OK(proxy->AddExactlyOnce(req, &resp, &controller));
+    }
+
+    CalculatorServiceProxy* const proxy;
+    const uint64_t client_sleep_for_ms;
+    RpcController controller;
+    ExactlyOnceRequestPB req;
+    ExactlyOnceResponsePB resp;
+    scoped_refptr<kudu::Thread> thread;
+  };
+
+
+  void CheckValueMatches(int expected_value) {
+    RpcController controller;
+    ExactlyOnceRequestPB req;
+    req.set_value_to_add(0);
+    ExactlyOnceResponsePB resp;
+    RequestTracker::SequenceNumber seq_no;
+    CHECK_OK(request_tracker_->NewSeqNo(&seq_no));
+    AddRequestId(&controller, kClientId, seq_no, 0);
+    ASSERT_OK(proxy_->AddExactlyOnce(req, &resp, &controller));
+    ASSERT_EQ(resp.current_val(), expected_value);
+    request_tracker_->RpcCompleted(seq_no);
+  }
+
+
+  // This continuously issues calls to the server, that often last longer than
+  // 'remember_responses_ttl_ms', making sure that we don't get errors back.
+  void DoLongWritesThread(MonoDelta run_for) {
+    MonoTime run_until = MonoTime::Now();
+    run_until.AddDelta(run_for);
+    int counter = 0;
+    while (MonoTime::Now() < run_until) {
+      unique_ptr<RetriableRpcExactlyOnceAdder> adder(new RetriableRpcExactlyOnceAdder(
+          test_picker_, request_tracker_, client_messenger_, 1,
+          rand() % (2 * FLAGS_remember_responses_ttl_ms)));
+
+      // This thread is used in the stress test where we're constantly running GC.
+      // So, once we get a "success" response, it's likely that the result will be
+      // GCed on the server side, and thus it's not safe to spuriously retry.
+      adder->rpc_->sometimes_retry_successful_ = false;
+      adder->SleepAndSend();
+      SleepFor(MonoDelta::FromMilliseconds(rand() % 10));
+      counter++;
+    }
+    ExactlyOnceResponsePB response;
+    ResultTracker::SequenceNumber sequence_number;
+    CHECK_OK(request_tracker_->NewSeqNo(&sequence_number));
+    CHECK_OK(MakeAddCall(sequence_number, 0, &response));
+    CHECK_EQ(response.current_val(), counter);
+    request_tracker_->RpcCompleted(sequence_number);
+  }
+
+  // Stubbornly sends the same request to the server, this should observe three states.
+  // The request should be successful at first, then its result should be GCed and the
+  // client should be GCed.
+  void StubbornlyWriteTheSameRequestThread(ResultTracker::SequenceNumber sequence_number,
+                                           MonoDelta run_for) {
+    MonoTime run_until = MonoTime::Now();
+    run_until.AddDelta(run_for);
+    // Make an initial request, so that we get a response to compare to.
+    ExactlyOnceResponsePB original_response;
+    CHECK_OK(MakeAddCall(sequence_number, 0, &original_response));
+
+    // Now repeat the same request. At first we should get the same response, then the result
+    // should be GCed and we should get STALE back. Finally the request should succeed again
+    // but we should get a new response.
+    bool result_gced = false;
+    bool client_gced = false;
+    while (MonoTime::Now() < run_until) {
+      ExactlyOnceResponsePB response;
+      Status s = MakeAddCall(sequence_number, 0, &response);
+      if (s.ok()) {
+        if (!result_gced) {
+          CHECK_EQ(SecureDebugString(response), SecureDebugString(original_response));
+        } else {
+          client_gced = true;
+          CHECK_NE(SecureDebugString(response), SecureDebugString(original_response));
+        }
+        SleepFor(MonoDelta::FromMilliseconds(rand() % 10));
+      } else if (s.IsRemoteError()) {
+        result_gced = true;
+        SleepFor(MonoDelta::FromMilliseconds(FLAGS_remember_clients_ttl_ms * 2));
+      }
+    }
+    CHECK(result_gced);
+    CHECK(client_gced);
+  }
+
+  Status MakeAddCall(ResultTracker::SequenceNumber sequence_number,
+                     int value_to_add,
+                     ExactlyOnceResponsePB* response,
+                     int attempt_no = -1) {
+    RpcController controller;
+    ExactlyOnceRequestPB req;
+    req.set_value_to_add(value_to_add);
+    if (attempt_no == -1) attempt_no = attempt_nos_.fetch_add(1);
+    AddRequestId(&controller, kClientId, sequence_number, attempt_no);
+    Status s = proxy_->AddExactlyOnce(req, response, &controller);
+    return s;
+  }
+
+ protected:
+  Sockaddr server_addr_;
+  atomic_int attempt_nos_;
+  shared_ptr<Messenger> client_messenger_;
+  std::unique_ptr<CalculatorServiceProxy> proxy_;
+  scoped_refptr<TestServerPicker> test_picker_;
+  scoped_refptr<RequestTracker> request_tracker_;
+};
+
+// Tests that we get exactly once semantics on RPCs when we send a bunch of requests with the
+// same sequence number as previous requests.
+TEST_F(ExactlyOnceRpcTest, TestExactlyOnceSemanticsAfterRpcCompleted) {
+  ASSERT_OK(StartServer());
+  ExactlyOnceResponsePB original_resp;
+  int mem_consumption = mem_tracker_->consumption();
+  {
+    RpcController controller;
+    ExactlyOnceRequestPB req;
+    req.set_value_to_add(1);
+
+    // Assign id 0.
+    AddRequestId(&controller, kClientId, 0, 0);
+
+    // Send the request the first time.
+    ASSERT_OK(proxy_->AddExactlyOnce(req, &original_resp, &controller));
+
+    // The incremental usage of a new client is the size of the response itself
+    // plus some fixed overhead for the client-tracking structure.
+    int expected_incremental_usage = original_resp.SpaceUsed() + 200;
+
+    int mem_consumption_after = mem_tracker_->consumption();
+    ASSERT_GT(mem_consumption_after - mem_consumption, expected_incremental_usage);
+    mem_consumption = mem_consumption_after;
+  }
+
+  // Now repeat the rpc 10 times, using the same sequence number, none of these should be executed
+  // and they should get the same response back.
+  for (int i = 0; i < 10; i++) {
+    RpcController controller;
+    controller.set_timeout(MonoDelta::FromSeconds(20));
+    ExactlyOnceRequestPB req;
+    req.set_value_to_add(1);
+    ExactlyOnceResponsePB resp;
+    AddRequestId(&controller, kClientId, 0, i + 1);
+    ASSERT_OK(proxy_->AddExactlyOnce(req, &resp, &controller));
+    ASSERT_EQ(resp.current_val(), 1);
+    ASSERT_EQ(resp.current_time_micros(), original_resp.current_time_micros());
+    // Sleep to give the MemTracker time to update -- we don't expect any update,
+    // but if we had a bug here, we'd only see it with this sleep.
+    SleepFor(MonoDelta::FromMilliseconds(100));
+    // We shouldn't have consumed any more memory since the responses were cached.
+    ASSERT_EQ(mem_consumption, mem_tracker_->consumption());
+  }
+
+  // Making a new request, from a new client, should double the memory consumption.
+  {
+    RpcController controller;
+    ExactlyOnceRequestPB req;
+    ExactlyOnceResponsePB resp;
+    req.set_value_to_add(1);
+
+    // Assign id 0.
+    AddRequestId(&controller, "test-client2", 0, 0);
+
+    // Send the first request for this new client.
+    ASSERT_OK(proxy_->AddExactlyOnce(req, &resp, &controller));
+    ASSERT_EQ(mem_tracker_->consumption(), mem_consumption * 2);
+  }
+}
+
+// Performs a series of requests in which each single request is attempted multiple times, as
+// the server side is instructed to spuriously fail attempts.
+// In CalculatorServiceRpc we sure that the same response is returned by all retries and,
+// after all the rpcs are done, we make sure that final result is the expected one.
+TEST_F(ExactlyOnceRpcTest, TestExactlyOnceSemanticsWithReplicatedRpc) {
+  ASSERT_OK(StartServer());
+  int kNumIterations = 10;
+  int kNumRpcs = 10;
+
+  if (AllowSlowTests()) {
+    kNumIterations = 100;
+    kNumRpcs = 100;
+  }
+
+  int count = 0;
+  for (int i = 0; i < kNumIterations; i ++) {
+    vector<unique_ptr<RetriableRpcExactlyOnceAdder>> adders;
+    for (int j = 0; j < kNumRpcs; j++) {
+      unique_ptr<RetriableRpcExactlyOnceAdder> adder(
+          new RetriableRpcExactlyOnceAdder(test_picker_, request_tracker_, client_messenger_, j));
+      adders.push_back(std::move(adder));
+      adders[j]->Start();
+      count += j;
+    }
+    for (int j = 0; j < kNumRpcs; j++) {
+      CHECK_OK(ThreadJoiner(adders[j]->thread.get()).Join());
+    }
+    CheckValueMatches(count);
+  }
+}
+
+// Performs a series of requests in which each single request is attempted by multiple threads.
+// On each iteration, after all the threads complete, we expect that the add operation was
+// executed exactly once.
+TEST_F(ExactlyOnceRpcTest, TestExactlyOnceSemanticsWithConcurrentUpdaters) {
+  ASSERT_OK(StartServer());
+  int kNumIterations = 10;
+  int kNumThreads = 10;
+
+  if (AllowSlowTests()) {
+    kNumIterations = 100;
+    kNumThreads = 100;
+  }
+
+  ResultTracker::SequenceNumber sequence_number = 0;
+  int memory_consumption_initial = mem_tracker_->consumption();
+  int single_response_size = 0;
+
+  // Measure memory consumption for a single response from the same client.
+  ExactlyOnceResponsePB resp;
+  ASSERT_OK(MakeAddCall(sequence_number, 1, &resp));
+
+  for (int i = 1; i <= kNumIterations; i ++) {
+    vector<unique_ptr<SimultaneousExactlyOnceAdder>> adders;
+    for (int j = 0; j < kNumThreads; j++) {
+      unique_ptr<SimultaneousExactlyOnceAdder> adder(
+          new SimultaneousExactlyOnceAdder(proxy_.get(),
+                                           i, // sequence number
+                                           1, // value
+                                           rand() % 20, // client_sleep
+                                           rand() % 10, // server_sleep
+                                           attempt_nos_.fetch_add(1))); // attempt number
+      adders.push_back(std::move(adder));
+      adders[j]->Start();
+    }
+    uint64_t time_micros = 0;
+    for (int j = 0; j < kNumThreads; j++) {
+      CHECK_OK(ThreadJoiner(adders[j]->thread.get()).Join());
+      ASSERT_EQ(adders[j]->resp.current_val(), i + 1);
+      if (time_micros == 0) {
+        time_micros = adders[j]->resp.current_time_micros();
+      } else {
+        ASSERT_EQ(adders[j]->resp.current_time_micros(), time_micros);
+      }
+    }
+
+    // After all adders finished we should at least the size of one more response.
+    // The actual size depends of multiple factors, for instance, how many calls were "attached"
+    // (which is timing dependent) so we can't be more precise than this.
+    ASSERT_GT(mem_tracker_->consumption(),
+              memory_consumption_initial + single_response_size * i);
+  }
+}
+
+TEST_F(ExactlyOnceRpcTest, TestExactlyOnceSemanticsGarbageCollection) {
+  FLAGS_remember_clients_ttl_ms = 500;
+  FLAGS_remember_responses_ttl_ms = 100;
+
+  ASSERT_OK(StartServer());
+
+  // Make a request.
+  ExactlyOnceResponsePB original;
+  ResultTracker::SequenceNumber sequence_number = 0;
+  ASSERT_OK(MakeAddCall(sequence_number, 1, &original));
+
+  // Making the same request again, should return the same response.
+  ExactlyOnceResponsePB resp;
+  ASSERT_OK(MakeAddCall(sequence_number, 1, &resp));
+  ASSERT_EQ(SecureShortDebugString(original), SecureShortDebugString(resp));
+
+  // Now sleep for 'remember_responses_ttl_ms' and run GC, we should then
+  // get a STALE back.
+  SleepFor(MonoDelta::FromMilliseconds(FLAGS_remember_responses_ttl_ms));
+  int64_t memory_consumption = mem_tracker_->consumption();
+  result_tracker_->GCResults();
+  ASSERT_LT(mem_tracker_->consumption(), memory_consumption);
+
+  resp.Clear();
+  Status s = MakeAddCall(sequence_number, 1, &resp);
+  ASSERT_TRUE(s.IsRemoteError());
+  ASSERT_STR_CONTAINS(s.ToString(), "is stale");
+
+  // Sleep again, this time for 'remember_clients_ttl_ms' and run GC again.
+  // The request should be successful, but its response should be a new one.
+  SleepFor(MonoDelta::FromMilliseconds(FLAGS_remember_clients_ttl_ms));
+  memory_consumption = mem_tracker_->consumption();
+  result_tracker_->GCResults();
+  ASSERT_LT(mem_tracker_->consumption(), memory_consumption);
+
+  resp.Clear();
+  ASSERT_OK(MakeAddCall(sequence_number, 1, &resp));
+  ASSERT_NE(SecureShortDebugString(resp), SecureShortDebugString(original));
+}
+
+// This test creates a thread continuously making requests to the server, some lasting longer
+// than the GC period, at the same time it runs GC, making sure that the corresponding
+// CompletionRecords/ClientStates are not deleted from underneath the ongoing requests.
+// This also creates a thread that runs GC very frequently and another thread that sends the
+// same request over and over and observes the possible states: request is ok, request is stale
+// request is ok again (because the client was forgotten).
+TEST_F(ExactlyOnceRpcTest, TestExactlyOnceSemanticsGarbageCollectionStressTest) {
+  FLAGS_remember_clients_ttl_ms = 100;
+  FLAGS_remember_responses_ttl_ms = 10;
+  FLAGS_result_tracker_gc_interval_ms = 10;
+
+  ASSERT_OK(StartServer());
+
+  // The write thread runs for a shorter period to make sure client GC has a
+  // chance to run.
+  MonoDelta writes_run_for = MonoDelta::FromSeconds(2);
+  MonoDelta stubborn_run_for = MonoDelta::FromSeconds(3);
+  if (AllowSlowTests()) {
+    writes_run_for = MonoDelta::FromSeconds(10);
+    stubborn_run_for = MonoDelta::FromSeconds(11);
+  }
+
+  result_tracker_->StartGCThread();
+
+  // Assign the first sequence number (0) to the 'stubborn writes' thread.
+  // This thread will keep making RPCs with this sequence number while
+  // the 'write_thread' will make normal requests with increasing sequence
+  // numbers.
+  ResultTracker::SequenceNumber stubborn_req_seq_num;
+  CHECK_OK(request_tracker_->NewSeqNo(&stubborn_req_seq_num));
+  ASSERT_EQ(stubborn_req_seq_num, 0);
+
+  scoped_refptr<kudu::Thread> stubborn_thread;
+  CHECK_OK(kudu::Thread::Create(
+      "stubborn", "stubborn", &ExactlyOnceRpcTest::StubbornlyWriteTheSameRequestThread,
+      this, stubborn_req_seq_num, stubborn_run_for, &stubborn_thread));
+
+  scoped_refptr<kudu::Thread> write_thread;
+  CHECK_OK(kudu::Thread::Create(
+      "write", "write", &ExactlyOnceRpcTest::DoLongWritesThread,
+      this, writes_run_for, &write_thread));
+
+  write_thread->Join();
+  stubborn_thread->Join();
+
+  // Within a few seconds, the consumption should be back to zero.
+  // Really, this should be within 100ms, but we'll give it a bit of
+  // time to avoid test flakiness.
+  AssertEventually([&]() {
+      ASSERT_EQ(0, mem_tracker_->consumption());
+    }, MonoDelta::FromSeconds(5));
+  NO_PENDING_FATALS();
+}
+
+
+} // namespace rpc
+} // namespace kudu
diff --git a/be/src/kudu/rpc/inbound_call.cc b/be/src/kudu/rpc/inbound_call.cc
new file mode 100644
index 0000000..6920071
--- /dev/null
+++ b/be/src/kudu/rpc/inbound_call.cc
@@ -0,0 +1,345 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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.
+
+#include "kudu/rpc/inbound_call.h"
+
+#include <cstdint>
+#include <memory>
+#include <ostream>
+
+#include <glog/logging.h>
+#include <google/protobuf/message.h>
+#include <google/protobuf/message_lite.h>
+
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/rpc/connection.h"
+#include "kudu/rpc/rpc_introspection.pb.h"
+#include "kudu/rpc/rpc_sidecar.h"
+#include "kudu/rpc/rpcz_store.h"
+#include "kudu/rpc/serialization.h"
+#include "kudu/rpc/service_if.h"
+#include "kudu/rpc/transfer.h"
+#include "kudu/util/debug/trace_event.h"
+#include "kudu/util/metrics.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/trace.h"
+
+namespace google {
+namespace protobuf {
+class FieldDescriptor;
+}
+}
+
+using google::protobuf::FieldDescriptor;
+using google::protobuf::Message;
+using google::protobuf::MessageLite;
+using std::string;
+using std::unique_ptr;
+using std::vector;
+using strings::Substitute;
+
+namespace kudu {
+namespace rpc {
+
+InboundCall::InboundCall(Connection* conn)
+  : conn_(conn),
+    trace_(new Trace),
+    method_info_(nullptr),
+    deadline_(MonoTime::Max()) {
+  RecordCallReceived();
+}
+
+InboundCall::~InboundCall() {}
+
+Status InboundCall::ParseFrom(gscoped_ptr<InboundTransfer> transfer) {
+  TRACE_EVENT_FLOW_BEGIN0("rpc", "InboundCall", this);
+  TRACE_EVENT0("rpc", "InboundCall::ParseFrom");
+  RETURN_NOT_OK(serialization::ParseMessage(transfer->data(), &header_, &serialized_request_));
+
+  // Adopt the service/method info from the header as soon as it's available.
+  if (PREDICT_FALSE(!header_.has_remote_method())) {
+    return Status::Corruption("Non-connection context request header must specify remote_method");
+  }
+  if (PREDICT_FALSE(!header_.remote_method().IsInitialized())) {
+    return Status::Corruption("remote_method in request header is not initialized",
+                              header_.remote_method().InitializationErrorString());
+  }
+  remote_method_.FromPB(header_.remote_method());
+
+  // Compute and cache the call deadline.
+  if (header_.has_timeout_millis() && header_.timeout_millis() != 0) {
+    deadline_ = timing_.time_received + MonoDelta::FromMilliseconds(header_.timeout_millis());
+  }
+
+  if (header_.sidecar_offsets_size() > TransferLimits::kMaxSidecars) {
+    return Status::Corruption(strings::Substitute(
+            "Received $0 additional payload slices, expected at most %d",
+            header_.sidecar_offsets_size(), TransferLimits::kMaxSidecars));
+  }
+
+  RETURN_NOT_OK(RpcSidecar::ParseSidecars(
+          header_.sidecar_offsets(), serialized_request_, inbound_sidecar_slices_));
+  if (header_.sidecar_offsets_size() > 0) {
+    // Trim the request to just the message
+    serialized_request_ = Slice(serialized_request_.data(), header_.sidecar_offsets(0));
+  }
+
+  // Retain the buffer that we have a view into.
+  transfer_.swap(transfer);
+  return Status::OK();
+}
+
+void InboundCall::RespondSuccess(const MessageLite& response) {
+  TRACE_EVENT0("rpc", "InboundCall::RespondSuccess");
+  Respond(response, true);
+}
+
+void InboundCall::RespondUnsupportedFeature(const vector<uint32_t>& unsupported_features) {
+  TRACE_EVENT0("rpc", "InboundCall::RespondUnsupportedFeature");
+  ErrorStatusPB err;
+  err.set_message("unsupported feature flags");
+  err.set_code(ErrorStatusPB::ERROR_INVALID_REQUEST);
+  for (uint32_t feature : unsupported_features) {
+    err.add_unsupported_feature_flags(feature);
+  }
+
+  Respond(err, false);
+}
+
+void InboundCall::RespondFailure(ErrorStatusPB::RpcErrorCodePB error_code,
+                                 const Status& status) {
+  TRACE_EVENT0("rpc", "InboundCall::RespondFailure");
+  ErrorStatusPB err;
+  err.set_message(status.ToString());
+  err.set_code(error_code);
+
+  Respond(err, false);
+}
+
+void InboundCall::RespondApplicationError(int error_ext_id, const std::string& message,
+                                          const MessageLite& app_error_pb) {
+  ErrorStatusPB err;
+  ApplicationErrorToPB(error_ext_id, message, app_error_pb, &err);
+  Respond(err, false);
+}
+
+void InboundCall::ApplicationErrorToPB(int error_ext_id, const std::string& message,
+                                       const google::protobuf::MessageLite& app_error_pb,
+                                       ErrorStatusPB* err) {
+  err->set_message(message);
+  const FieldDescriptor* app_error_field =
+    err->GetReflection()->FindKnownExtensionByNumber(error_ext_id);
+  if (app_error_field != nullptr) {
+    err->GetReflection()->MutableMessage(err, app_error_field)->CheckTypeAndMergeFrom(app_error_pb);
+  } else {
+    LOG(DFATAL) << "Unable to find application error extension ID " << error_ext_id
+                << " (message=" << message << ")";
+  }
+}
+
+void InboundCall::Respond(const MessageLite& response,
+                          bool is_success) {
+  TRACE_EVENT_FLOW_END0("rpc", "InboundCall", this);
+  SerializeResponseBuffer(response, is_success);
+
+  TRACE_EVENT_ASYNC_END1("rpc", "InboundCall", this,
+                         "method", remote_method_.method_name());
+  TRACE_TO(trace_, "Queueing $0 response", is_success ? "success" : "failure");
+  RecordHandlingCompleted();
+  conn_->rpcz_store()->AddCall(this);
+  conn_->QueueResponseForCall(gscoped_ptr<InboundCall>(this));
+}
+
+void InboundCall::SerializeResponseBuffer(const MessageLite& response,
+                                          bool is_success) {
+  if (PREDICT_FALSE(!response.IsInitialized())) {
+    LOG(ERROR) << "Invalid RPC response for " << ToString()
+               << ": protobuf missing required fields: "
+               << response.InitializationErrorString();
+    // Send it along anyway -- the client will also notice the missing fields
+    // and produce an error on the other side, but this will at least
+    // make it clear on both sides of the RPC connection what kind of error
+    // happened.
+  }
+
+  uint32_t protobuf_msg_size = response.ByteSize();
+
+  ResponseHeader resp_hdr;
+  resp_hdr.set_call_id(header_.call_id());
+  resp_hdr.set_is_error(!is_success);
+  int32_t sidecar_byte_size = 0;
+  for (const unique_ptr<RpcSidecar>& car : outbound_sidecars_) {
+    resp_hdr.add_sidecar_offsets(sidecar_byte_size + protobuf_msg_size);
+    int32_t sidecar_bytes = car->AsSlice().size();
+    DCHECK_LE(sidecar_byte_size, TransferLimits::kMaxTotalSidecarBytes - sidecar_bytes);
+    sidecar_byte_size += sidecar_bytes;
+  }
+
+  serialization::SerializeMessage(response, &response_msg_buf_,
+                                  sidecar_byte_size, true);
+  int64_t main_msg_size = sidecar_byte_size + response_msg_buf_.size();
+  serialization::SerializeHeader(resp_hdr, main_msg_size,
+                                 &response_hdr_buf_);
+}
+
+size_t InboundCall::SerializeResponseTo(TransferPayload* slices) const {
+  TRACE_EVENT0("rpc", "InboundCall::SerializeResponseTo");
+  DCHECK_GT(response_hdr_buf_.size(), 0);
+  DCHECK_GT(response_msg_buf_.size(), 0);
+  size_t n_slices = 2 + outbound_sidecars_.size();
+  DCHECK_LE(n_slices, slices->size());
+  auto slice_iter = slices->begin();
+  *slice_iter++ = Slice(response_hdr_buf_);
+  *slice_iter++ = Slice(response_msg_buf_);
+  for (auto& sidecar : outbound_sidecars_) {
+    *slice_iter++ = sidecar->AsSlice();
+  }
+  DCHECK_EQ(slice_iter - slices->begin(), n_slices);
+  return n_slices;
+}
+
+Status InboundCall::AddOutboundSidecar(unique_ptr<RpcSidecar> car, int* idx) {
+  // Check that the number of sidecars does not exceed the number of payload
+  // slices that are free (two are used up by the header and main message
+  // protobufs).
+  if (outbound_sidecars_.size() > TransferLimits::kMaxSidecars) {
+    return Status::ServiceUnavailable("All available sidecars already used");
+  }
+  int64_t sidecar_bytes = car->AsSlice().size();
+  if (outbound_sidecars_total_bytes_ >
+      TransferLimits::kMaxTotalSidecarBytes - sidecar_bytes) {
+    return Status::RuntimeError(Substitute("Total size of sidecars $0 would exceed limit $1",
+        static_cast<int64_t>(outbound_sidecars_total_bytes_) + sidecar_bytes,
+        TransferLimits::kMaxTotalSidecarBytes));
+  }
+
+  outbound_sidecars_.emplace_back(std::move(car));
+  outbound_sidecars_total_bytes_ += sidecar_bytes;
+  DCHECK_GE(outbound_sidecars_total_bytes_, 0);
+  *idx = outbound_sidecars_.size() - 1;
+  return Status::OK();
+}
+
+string InboundCall::ToString() const {
+  if (header_.has_request_id()) {
+    return Substitute("Call $0 from $1 (ReqId={client: $2, seq_no=$3, attempt_no=$4})",
+                      remote_method_.ToString(),
+                      conn_->remote().ToString(),
+                      header_.request_id().client_id(),
+                      header_.request_id().seq_no(),
+                      header_.request_id().attempt_no());
+  }
+  return Substitute("Call $0 from $1 (request call id $2)",
+                      remote_method_.ToString(),
+                      conn_->remote().ToString(),
+                      header_.call_id());
+}
+
+void InboundCall::DumpPB(const DumpRunningRpcsRequestPB& req,
+                         RpcCallInProgressPB* resp) {
+  resp->mutable_header()->CopyFrom(header_);
+  if (req.include_traces() && trace_) {
+    resp->set_trace_buffer(trace_->DumpToString());
+  }
+  resp->set_micros_elapsed((MonoTime::Now() - timing_.time_received)
+                           .ToMicroseconds());
+}
+
+const RemoteUser& InboundCall::remote_user() const {
+  return conn_->remote_user();
+}
+
+const Sockaddr& InboundCall::remote_address() const {
+  return conn_->remote();
+}
+
+const scoped_refptr<Connection>& InboundCall::connection() const {
+  return conn_;
+}
+
+Trace* InboundCall::trace() {
+  return trace_.get();
+}
+
+void InboundCall::RecordCallReceived() {
+  TRACE_EVENT_ASYNC_BEGIN0("rpc", "InboundCall", this);
+  DCHECK(!timing_.time_received.Initialized());  // Protect against multiple calls.
+  timing_.time_received = MonoTime::Now();
+}
+
+void InboundCall::RecordHandlingStarted(Histogram* incoming_queue_time) {
+  DCHECK(incoming_queue_time != nullptr);
+  DCHECK(!timing_.time_handled.Initialized());  // Protect against multiple calls.
+  timing_.time_handled = MonoTime::Now();
+  incoming_queue_time->Increment(
+      (timing_.time_handled - timing_.time_received).ToMicroseconds());
+}
+
+void InboundCall::RecordHandlingCompleted() {
+  DCHECK(!timing_.time_completed.Initialized());  // Protect against multiple calls.
+  timing_.time_completed = MonoTime::Now();
+
+  if (!timing_.time_handled.Initialized()) {
+    // Sometimes we respond to a call before we begin handling it (e.g. due to queue
+    // overflow, etc). These cases should not be counted against the histogram.
+    return;
+  }
+
+  if (method_info_) {
+    method_info_->handler_latency_histogram->Increment(
+        (timing_.time_completed - timing_.time_handled).ToMicroseconds());
+  }
+}
+
+bool InboundCall::ClientTimedOut() const {
+  return MonoTime::Now() >= deadline_;
+}
+
+MonoTime InboundCall::GetTimeReceived() const {
+  return timing_.time_received;
+}
+
+vector<uint32_t> InboundCall::GetRequiredFeatures() const {
+  vector<uint32_t> features;
+  for (uint32_t feature : header_.required_feature_flags()) {
+    features.push_back(feature);
+  }
+  return features;
+}
+
+Status InboundCall::GetInboundSidecar(int idx, Slice* sidecar) const {
+  DCHECK(transfer_) << "Sidecars have been discarded";
+  if (idx < 0 || idx >= header_.sidecar_offsets_size()) {
+    return Status::InvalidArgument(strings::Substitute(
+            "Index $0 does not reference a valid sidecar", idx));
+  }
+  *sidecar = inbound_sidecar_slices_[idx];
+  return Status::OK();
+}
+
+void InboundCall::DiscardTransfer() {
+  transfer_.reset();
+}
+
+size_t InboundCall::GetTransferSize() {
+  if (!transfer_) return 0;
+  return transfer_->data().size();
+}
+
+} // namespace rpc
+} // namespace kudu
diff --git a/be/src/kudu/rpc/inbound_call.h b/be/src/kudu/rpc/inbound_call.h
new file mode 100644
index 0000000..0db4c37
--- /dev/null
+++ b/be/src/kudu/rpc/inbound_call.h
@@ -0,0 +1,286 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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.
+#ifndef KUDU_RPC_INBOUND_CALL_H
+#define KUDU_RPC_INBOUND_CALL_H
+
+#include <cstddef>
+#include <cstdint>
+#include <memory>
+#include <ostream>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/rpc/remote_method.h"
+#include "kudu/rpc/rpc_header.pb.h"
+#include "kudu/rpc/service_if.h"
+#include "kudu/rpc/transfer.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+
+namespace google {
+namespace protobuf {
+class MessageLite;
+} // namespace protobuf
+} // namespace google
+
+namespace kudu {
+
+class Histogram;
+class Sockaddr;
+class Trace;
+
+namespace rpc {
+
+class Connection;
+class DumpRunningRpcsRequestPB;
+class RemoteUser;
+class RpcCallInProgressPB;
+class RpcSidecar;
+
+struct InboundCallTiming {
+  MonoTime time_received;   // Time the call was first accepted.
+  MonoTime time_handled;    // Time the call handler was kicked off.
+  MonoTime time_completed;  // Time the call handler completed.
+
+  MonoDelta TotalDuration() const {
+    return time_completed - time_received;
+  }
+};
+
+// Inbound call on server
+class InboundCall {
+ public:
+  explicit InboundCall(Connection* conn);
+  ~InboundCall();
+
+  // Parse an inbound call message.
+  //
+  // This only deserializes the call header, populating the 'header_' and
+  // 'serialized_request_' member variables. The actual call parameter is
+  // not deserialized, as this may be CPU-expensive, and this is called
+  // from the reactor thread.
+  Status ParseFrom(gscoped_ptr<InboundTransfer> transfer);
+
+  // Return the serialized request parameter protobuf.
+  const Slice& serialized_request() const {
+    DCHECK(transfer_) << "Transfer discarded before parameter parsing";
+    return serialized_request_;
+  }
+
+  const RemoteMethod& remote_method() const {
+    return remote_method_;
+  }
+
+  const int32_t call_id() const {
+    return header_.call_id();
+  }
+
+  // Serializes 'response' into the InboundCall's internal buffer, and marks
+  // the call as a success. Enqueues the response back to the connection
+  // that made the call.
+  //
+  // This method deletes the InboundCall object, so no further calls may be
+  // made after this one.
+  void RespondSuccess(const google::protobuf::MessageLite& response);
+
+  // Serializes a failure response into the internal buffer, marking the
+  // call as a failure. Enqueues the response back to the connection that
+  // made the call.
+  //
+  // This method deletes the InboundCall object, so no further calls may be
+  // made after this one.
+  void RespondFailure(ErrorStatusPB::RpcErrorCodePB error_code,
+                      const Status &status);
+
+  void RespondUnsupportedFeature(const std::vector<uint32_t>& unsupported_features);
+
+  void RespondApplicationError(int error_ext_id, const std::string& message,
+                               const google::protobuf::MessageLite& app_error_pb);
+
+  // Convert an application error extension to an ErrorStatusPB.
+  // These ErrorStatusPB objects are what are returned in application error responses.
+  static void ApplicationErrorToPB(int error_ext_id, const std::string& message,
+                                   const google::protobuf::MessageLite& app_error_pb,
+                                   ErrorStatusPB* err);
+
+  // Serialize the response packet for the finished call into 'slices'.
+  // The resulting slices refer to memory in this object.
+  // Returns the number of slices in the serialized response.
+  size_t SerializeResponseTo(TransferPayload* slices) const;
+
+  // See RpcContext::AddRpcSidecar()
+  Status AddOutboundSidecar(std::unique_ptr<RpcSidecar> car, int* idx);
+
+  std::string ToString() const;
+
+  void DumpPB(const DumpRunningRpcsRequestPB& req, RpcCallInProgressPB* resp);
+
+  const RemoteUser& remote_user() const;
+
+  const Sockaddr& remote_address() const;
+
+  const scoped_refptr<Connection>& connection() const;
+
+  Trace* trace();
+
+  const InboundCallTiming& timing() const {
+    return timing_;
+  }
+
+  const RequestHeader& header() const {
+    return header_;
+  }
+
+  // Associate this call with a particular method that will be invoked
+  // by the service.
+  void set_method_info(scoped_refptr<RpcMethodInfo> info) {
+    method_info_ = std::move(info);
+  }
+
+  // Return the method associated with this call. This is set just before
+  // the call is enqueued onto the service queue, and therefore may be
+  // 'nullptr' for much of the lifecycle of a call.
+  RpcMethodInfo* method_info() {
+    return method_info_.get();
+  }
+
+  // When this InboundCall was received (instantiated).
+  // Should only be called once on a given instance.
+  // Not thread-safe. Should only be called by the current "owner" thread.
+  void RecordCallReceived();
+
+  // When RPC call Handle() was called on the server side.
+  // Updates the Histogram with time elapsed since the call was received,
+  // and should only be called once on a given instance.
+  // Not thread-safe. Should only be called by the current "owner" thread.
+  void RecordHandlingStarted(Histogram* incoming_queue_time);
+
+  // Return true if the deadline set by the client has already elapsed.
+  // In this case, the server may stop processing the call, since the
+  // call response will be ignored anyway.
+  bool ClientTimedOut() const;
+
+  // Return an upper bound on the client timeout deadline. This does not
+  // account for transmission delays between the client and the server.
+  // If the client did not specify a deadline, returns MonoTime::Max().
+  MonoTime GetClientDeadline() const {
+    return deadline_;
+  }
+
+  // Return the time when this call was received.
+  MonoTime GetTimeReceived() const;
+
+  // Returns the set of application-specific feature flags required to service
+  // the RPC.
+  std::vector<uint32_t> GetRequiredFeatures() const;
+
+  // Get a sidecar sent as part of the request. If idx < 0 || idx > num sidecars - 1,
+  // returns an error.
+  Status GetInboundSidecar(int idx, Slice* sidecar) const;
+
+  // Releases the buffer that contains the request + sidecar data. It is an error to
+  // access sidecars or serialized_request() after this method is called.
+  void DiscardTransfer();
+
+  // Returns the size of the transfer buffer that backs this call. If the transfer does
+  // not exist (e.g. GetTransferSize() is called after DiscardTransfer()), returns 0.
+  size_t GetTransferSize();
+
+ private:
+  friend class RpczStore;
+
+  // Serialize and queue the response.
+  void Respond(const google::protobuf::MessageLite& response,
+               bool is_success);
+
+  // Serialize a response message for either success or failure. If it is a success,
+  // 'response' should be the user-defined response type for the call. If it is a
+  // failure, 'response' should be an ErrorStatusPB instance.
+  void SerializeResponseBuffer(const google::protobuf::MessageLite& response,
+                               bool is_success);
+
+  // When RPC call Handle() completed execution on the server side.
+  // Updates the Histogram with time elapsed since the call was started,
+  // and should only be called once on a given instance.
+  // Not thread-safe. Should only be called by the current "owner" thread.
+  void RecordHandlingCompleted();
+
+  // The connection on which this inbound call arrived.
+  scoped_refptr<Connection> conn_;
+
+  // The header of the incoming call. Set by ParseFrom()
+  RequestHeader header_;
+
+  // The serialized bytes of the request param protobuf. Set by ParseFrom().
+  // This references memory held by 'transfer_'.
+  Slice serialized_request_;
+
+  // The transfer that produced the call.
+  // This is kept around because it retains the memory referred to
+  // by 'serialized_request_' above.
+  gscoped_ptr<InboundTransfer> transfer_;
+
+  // The buffers for serialized response. Set by SerializeResponseBuffer().
+  faststring response_hdr_buf_;
+  faststring response_msg_buf_;
+
+  // Vector of additional sidecars that are tacked on to the call's response
+  // after serialization of the protobuf. See rpc/rpc_sidecar.h for more info.
+  std::vector<std::unique_ptr<RpcSidecar>> outbound_sidecars_;
+
+  // Total size of sidecars in outbound_sidecars_. This is limited to a maximum
+  // of TransferLimits::kMaxTotalSidecarBytes.
+  int32_t outbound_sidecars_total_bytes_ = 0;
+
+  // Inbound sidecars from the request. The slices are views onto transfer_. There are as
+  // many slices as header_.sidecar_offsets_size().
+  Slice inbound_sidecar_slices_[TransferLimits::kMaxSidecars];
+
+  // The trace buffer.
+  scoped_refptr<Trace> trace_;
+
+  // Timing information related to this RPC call.
+  InboundCallTiming timing_;
+
+  // Proto service this calls belongs to. Used for routing.
+  // This field is filled in when the inbound request header is parsed.
+  RemoteMethod remote_method_;
+
+  // After the method has been looked up within the service, this is filled in
+  // to point to the information about this method. Acts as a pointer back to
+  // per-method info such as tracing.
+  scoped_refptr<RpcMethodInfo> method_info_;
+
+  // A time at which the client will time out, or MonoTime::Max if the
+  // client did not pass a timeout.
+  MonoTime deadline_;
+
+  DISALLOW_COPY_AND_ASSIGN(InboundCall);
+};
+
+} // namespace rpc
+} // namespace kudu
+
+#endif
diff --git a/be/src/kudu/rpc/messenger.cc b/be/src/kudu/rpc/messenger.cc
new file mode 100644
index 0000000..17ac0c5
--- /dev/null
+++ b/be/src/kudu/rpc/messenger.cc
@@ -0,0 +1,502 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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.
+
+#include "kudu/rpc/messenger.h"
+
+#include <cstdlib>
+#include <functional>
+#include <mutex>
+#include <ostream>
+#include <string>
+#include <type_traits>
+#include <utility>
+
+#include <glog/logging.h>
+
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/map-util.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/stl_util.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/rpc/acceptor_pool.h"
+#include "kudu/rpc/connection_id.h"
+#include "kudu/rpc/inbound_call.h"
+#include "kudu/rpc/outbound_call.h"
+#include "kudu/rpc/reactor.h"
+#include "kudu/rpc/remote_method.h"
+#include "kudu/rpc/rpc_header.pb.h"
+#include "kudu/rpc/rpc_service.h"
+#include "kudu/rpc/rpcz_store.h"
+#include "kudu/rpc/sasl_common.h"
+#include "kudu/rpc/server_negotiation.h"
+#include "kudu/rpc/service_if.h"
+#include "kudu/security/openssl_util.h"
+#include "kudu/security/tls_context.h"
+#include "kudu/security/token_verifier.h"
+#include "kudu/util/flags.h"
+#include "kudu/util/metrics.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/net/socket.h"
+#include "kudu/util/scoped_cleanup.h"
+#include "kudu/util/status.h"
+#include "kudu/util/thread_restrictions.h"
+#include "kudu/util/threadpool.h"
+
+using std::string;
+using std::shared_ptr;
+using std::make_shared;
+using strings::Substitute;
+
+namespace boost {
+template <typename Signature> class function;
+}
+
+namespace kudu {
+namespace rpc {
+
+MessengerBuilder::MessengerBuilder(std::string name)
+    : name_(std::move(name)),
+      connection_keepalive_time_(MonoDelta::FromMilliseconds(65000)),
+      num_reactors_(4),
+      min_negotiation_threads_(0),
+      max_negotiation_threads_(4),
+      coarse_timer_granularity_(MonoDelta::FromMilliseconds(100)),
+      rpc_negotiation_timeout_ms_(3000),
+      sasl_proto_name_("kudu"),
+      rpc_authentication_("optional"),
+      rpc_encryption_("optional"),
+      rpc_tls_ciphers_(kudu::security::SecurityDefaults::kDefaultTlsCiphers),
+      rpc_tls_min_protocol_(kudu::security::SecurityDefaults::kDefaultTlsMinVersion),
+      enable_inbound_tls_(false),
+      reuseport_(false) {
+}
+
+MessengerBuilder& MessengerBuilder::set_connection_keepalive_time(const MonoDelta &keepalive) {
+  connection_keepalive_time_ = keepalive;
+  return *this;
+}
+
+MessengerBuilder& MessengerBuilder::set_num_reactors(int num_reactors) {
+  num_reactors_ = num_reactors;
+  return *this;
+}
+
+MessengerBuilder& MessengerBuilder::set_min_negotiation_threads(int min_negotiation_threads) {
+  min_negotiation_threads_ = min_negotiation_threads;
+  return *this;
+}
+
+MessengerBuilder& MessengerBuilder::set_max_negotiation_threads(int max_negotiation_threads) {
+  max_negotiation_threads_ = max_negotiation_threads;
+  return *this;
+}
+
+MessengerBuilder& MessengerBuilder::set_coarse_timer_granularity(const MonoDelta &granularity) {
+  coarse_timer_granularity_ = granularity;
+  return *this;
+}
+
+MessengerBuilder &MessengerBuilder::set_metric_entity(
+    const scoped_refptr<MetricEntity>& metric_entity) {
+  metric_entity_ = metric_entity;
+  return *this;
+}
+
+MessengerBuilder &MessengerBuilder::set_connection_keep_alive_time(int32_t time_in_ms) {
+  connection_keepalive_time_ = MonoDelta::FromMilliseconds(time_in_ms);
+  return *this;
+}
+
+MessengerBuilder &MessengerBuilder::set_rpc_negotiation_timeout_ms(int64_t time_in_ms) {
+  rpc_negotiation_timeout_ms_ = time_in_ms;
+  return *this;
+}
+
+MessengerBuilder &MessengerBuilder::set_sasl_proto_name(const std::string& sasl_proto_name) {
+  sasl_proto_name_ = sasl_proto_name;
+  return *this;
+}
+
+MessengerBuilder &MessengerBuilder::set_rpc_authentication(const std::string& rpc_authentication) {
+  rpc_authentication_ = rpc_authentication;
+  return *this;
+}
+
+MessengerBuilder &MessengerBuilder::set_rpc_encryption(const std::string& rpc_encryption) {
+  rpc_encryption_ = rpc_encryption;
+  return *this;
+}
+
+MessengerBuilder &MessengerBuilder::set_rpc_tls_ciphers(const std::string& rpc_tls_ciphers) {
+  rpc_tls_ciphers_ = rpc_tls_ciphers;
+  return *this;
+}
+
+MessengerBuilder &MessengerBuilder::set_rpc_tls_min_protocol(
+    const std::string& rpc_tls_min_protocol) {
+  rpc_tls_min_protocol_ = rpc_tls_min_protocol;
+  return *this;
+}
+
+MessengerBuilder& MessengerBuilder::set_epki_cert_key_files(
+    const std::string& cert, const std::string& private_key) {
+  rpc_certificate_file_ = cert;
+  rpc_private_key_file_ = private_key;
+  return *this;
+}
+
+MessengerBuilder& MessengerBuilder::set_epki_certificate_authority_file(const std::string& ca) {
+  rpc_ca_certificate_file_ = ca;
+  return *this;
+}
+
+MessengerBuilder& MessengerBuilder::set_epki_private_password_key_cmd(const std::string& cmd) {
+  rpc_private_key_password_cmd_ = cmd;
+  return *this;
+}
+
+MessengerBuilder& MessengerBuilder::set_keytab_file(const std::string& keytab_file) {
+  keytab_file_ = keytab_file;
+  return *this;
+}
+
+MessengerBuilder& MessengerBuilder::enable_inbound_tls() {
+  enable_inbound_tls_ = true;
+  return *this;
+}
+
+MessengerBuilder& MessengerBuilder::set_reuseport() {
+  reuseport_ = true;
+  return *this;
+}
+
+Status MessengerBuilder::Build(shared_ptr<Messenger> *msgr) {
+  // Initialize SASL library before we start making requests
+  RETURN_NOT_OK(SaslInit(!keytab_file_.empty()));
+
+  Messenger* new_msgr(new Messenger(*this));
+
+  auto cleanup = MakeScopedCleanup([&] () {
+      new_msgr->AllExternalReferencesDropped();
+  });
+
+  RETURN_NOT_OK(ParseTriState("--rpc_authentication",
+                              rpc_authentication_,
+                              &new_msgr->authentication_));
+
+  RETURN_NOT_OK(ParseTriState("--rpc_encryption",
+                              rpc_encryption_,
+                              &new_msgr->encryption_));
+
+  RETURN_NOT_OK(new_msgr->Init());
+  if (new_msgr->encryption_ != RpcEncryption::DISABLED && enable_inbound_tls_) {
+    auto* tls_context = new_msgr->mutable_tls_context();
+
+    if (!rpc_certificate_file_.empty()) {
+      CHECK(!rpc_private_key_file_.empty());
+      CHECK(!rpc_ca_certificate_file_.empty());
+
+      // TODO(KUDU-1920): should we try and enforce that the server
+      // is in the subject or alt names of the cert?
+      RETURN_NOT_OK(tls_context->LoadCertificateAuthority(rpc_ca_certificate_file_));
+      if (rpc_private_key_password_cmd_.empty()) {
+        RETURN_NOT_OK(tls_context->LoadCertificateAndKey(rpc_certificate_file_,
+                                                         rpc_private_key_file_));
+      } else {
+        RETURN_NOT_OK(tls_context->LoadCertificateAndPasswordProtectedKey(
+            rpc_certificate_file_, rpc_private_key_file_,
+            [&](){
+              string ret;
+              WARN_NOT_OK(security::GetPasswordFromShellCommand(
+                  rpc_private_key_password_cmd_, &ret),
+                  "could not get RPC password from configured command");
+              return ret;
+            }
+        ));
+      }
+    } else {
+      RETURN_NOT_OK(tls_context->GenerateSelfSignedCertAndKey());
+    }
+  }
+
+  // See docs on Messenger::retain_self_ for info about this odd hack.
+  cleanup.cancel();
+  *msgr = shared_ptr<Messenger>(new_msgr, std::mem_fun(&Messenger::AllExternalReferencesDropped));
+  return Status::OK();
+}
+
+// See comment on Messenger::retain_self_ member.
+void Messenger::AllExternalReferencesDropped() {
+  // The last external ref may have been dropped in the context of a task
+  // running on a reactor thread. If that's the case, a SYNC shutdown here
+  // would deadlock.
+  //
+  // If a SYNC shutdown is desired, Shutdown() should be called explicitly.
+  ShutdownInternal(ShutdownMode::ASYNC);
+
+  CHECK(retain_self_.get());
+  // If we have no more external references, then we no longer
+  // need to retain ourself. We'll destruct as soon as all our
+  // internal-facing references are dropped (ie those from reactor
+  // threads).
+  retain_self_.reset();
+}
+
+void Messenger::Shutdown() {
+  ShutdownInternal(ShutdownMode::SYNC);
+}
+
+void Messenger::ShutdownInternal(ShutdownMode mode) {
+  if (mode == ShutdownMode::SYNC) {
+    ThreadRestrictions::AssertWaitAllowed();
+  }
+
+  // Since we're shutting down, it's OK to block.
+  //
+  // TODO(adar): this ought to be removed (i.e. if ASYNC, waiting should be
+  // forbidden, and if SYNC, we already asserted above), but that's not
+  // possible while shutting down thread and acceptor pools still involves
+  // joining threads.
+  ThreadRestrictions::ScopedAllowWait allow_wait;
+
+  acceptor_vec_t pools_to_shutdown;
+  RpcServicesMap services_to_release;
+  {
+    std::lock_guard<percpu_rwlock> guard(lock_);
+    if (closing_) {
+      return;
+    }
+    VLOG(1) << "shutting down messenger " << name_;
+    closing_ = true;
+
+    services_to_release = std::move(rpc_services_);
+    pools_to_shutdown = std::move(acceptor_pools_);
+  }
+
+  // Destroy state outside of the lock.
+  services_to_release.clear();
+  for (const auto& p : pools_to_shutdown) {
+    p->Shutdown();
+  }
+
+  // Need to shut down negotiation pool before the reactors, since the
+  // reactors close the Connection sockets, and may race against the negotiation
+  // threads' blocking reads & writes.
+  client_negotiation_pool_->Shutdown();
+  server_negotiation_pool_->Shutdown();
+
+  for (Reactor* reactor : reactors_) {
+    reactor->Shutdown(mode);
+  }
+}
+
+Status Messenger::AddAcceptorPool(const Sockaddr &accept_addr,
+                                  shared_ptr<AcceptorPool>* pool) {
+  // Before listening, if we expect to require Kerberos, we want to verify
+  // that everything is set up correctly. This way we'll generate errors on
+  // startup rather than later on when we first receive a client connection.
+  if (!keytab_file_.empty()) {
+    RETURN_NOT_OK_PREPEND(ServerNegotiation::PreflightCheckGSSAPI(sasl_proto_name()),
+                          "GSSAPI/Kerberos not properly configured");
+  }
+
+  Socket sock;
+  RETURN_NOT_OK(sock.Init(0));
+  RETURN_NOT_OK(sock.SetReuseAddr(true));
+  if (reuseport_) {
+    RETURN_NOT_OK(sock.SetReusePort(true));
+  }
+  RETURN_NOT_OK(sock.Bind(accept_addr));
+  Sockaddr remote;
+  RETURN_NOT_OK(sock.GetSocketAddress(&remote));
+  auto acceptor_pool(make_shared<AcceptorPool>(this, &sock, remote));
+
+  std::lock_guard<percpu_rwlock> guard(lock_);
+  acceptor_pools_.push_back(acceptor_pool);
+  pool->swap(acceptor_pool);
+  return Status::OK();
+}
+
+// Register a new RpcService to handle inbound requests.
+Status Messenger::RegisterService(const string& service_name,
+                                  const scoped_refptr<RpcService>& service) {
+  DCHECK(service);
+  std::lock_guard<percpu_rwlock> guard(lock_);
+  if (InsertIfNotPresent(&rpc_services_, service_name, service)) {
+    return Status::OK();
+  } else {
+    return Status::AlreadyPresent("This service is already present");
+  }
+}
+
+void Messenger::UnregisterAllServices() {
+  RpcServicesMap to_release;
+  {
+    std::lock_guard<percpu_rwlock> guard(lock_);
+    to_release = std::move(rpc_services_);
+  }
+  // Release the map outside of the lock.
+}
+
+Status Messenger::UnregisterService(const string& service_name) {
+  scoped_refptr<RpcService> to_release;
+  {
+    std::lock_guard<percpu_rwlock> guard(lock_);
+    to_release = EraseKeyReturnValuePtr(&rpc_services_, service_name);
+    if (!to_release) {
+      return Status::ServiceUnavailable(Substitute(
+          "service $0 not registered on $1", service_name, name_));
+    }
+  }
+  // Release the service outside of the lock.
+  return Status::OK();
+}
+
+void Messenger::QueueOutboundCall(const shared_ptr<OutboundCall> &call) {
+  Reactor *reactor = RemoteToReactor(call->conn_id().remote());
+  reactor->QueueOutboundCall(call);
+}
+
+void Messenger::QueueInboundCall(gscoped_ptr<InboundCall> call) {
+  shared_lock<rw_spinlock> guard(lock_.get_lock());
+  scoped_refptr<RpcService>* service = FindOrNull(rpc_services_,
+                                                  call->remote_method().service_name());
+  if (PREDICT_FALSE(!service)) {
+    Status s =  Status::ServiceUnavailable(Substitute("service $0 not registered on $1",
+                                                      call->remote_method().service_name(), name_));
+    LOG(INFO) << s.ToString();
+    call.release()->RespondFailure(ErrorStatusPB::ERROR_NO_SUCH_SERVICE, s);
+    return;
+  }
+
+  call->set_method_info((*service)->LookupMethod(call->remote_method()));
+
+  // The RpcService will respond to the client on success or failure.
+  WARN_NOT_OK((*service)->QueueInboundCall(std::move(call)), "Unable to handle RPC call");
+}
+
+void Messenger::QueueCancellation(const shared_ptr<OutboundCall> &call) {
+  Reactor *reactor = RemoteToReactor(call->conn_id().remote());
+  reactor->QueueCancellation(call);
+}
+
+void Messenger::RegisterInboundSocket(Socket *new_socket, const Sockaddr &remote) {
+  Reactor *reactor = RemoteToReactor(remote);
+  reactor->RegisterInboundSocket(new_socket, remote);
+}
+
+Messenger::Messenger(const MessengerBuilder &bld)
+  : name_(bld.name_),
+    closing_(false),
+    authentication_(RpcAuthentication::REQUIRED),
+    encryption_(RpcEncryption::REQUIRED),
+    tls_context_(new security::TlsContext(bld.rpc_tls_ciphers_, bld.rpc_tls_min_protocol_)),
+    token_verifier_(new security::TokenVerifier()),
+    rpcz_store_(new RpczStore()),
+    metric_entity_(bld.metric_entity_),
+    rpc_negotiation_timeout_ms_(bld.rpc_negotiation_timeout_ms_),
+    sasl_proto_name_(bld.sasl_proto_name_),
+    keytab_file_(bld.keytab_file_),
+    reuseport_(bld.reuseport_),
+    retain_self_(this) {
+  for (int i = 0; i < bld.num_reactors_; i++) {
+    reactors_.push_back(new Reactor(retain_self_, i, bld));
+  }
+  CHECK_OK(ThreadPoolBuilder("client-negotiator")
+      .set_min_threads(bld.min_negotiation_threads_)
+      .set_max_threads(bld.max_negotiation_threads_)
+      .Build(&client_negotiation_pool_));
+  CHECK_OK(ThreadPoolBuilder("server-negotiator")
+      .set_min_threads(bld.min_negotiation_threads_)
+      .set_max_threads(bld.max_negotiation_threads_)
+      .Build(&server_negotiation_pool_));
+}
+
+Messenger::~Messenger() {
+  std::lock_guard<percpu_rwlock> guard(lock_);
+  CHECK(closing_) << "Should have already shut down";
+  STLDeleteElements(&reactors_);
+}
+
+Reactor* Messenger::RemoteToReactor(const Sockaddr &remote) {
+  uint32_t hashCode = remote.HashCode();
+  int reactor_idx = hashCode % reactors_.size();
+  // This is just a static partitioning; we could get a lot
+  // fancier with assigning Sockaddrs to Reactors.
+  return reactors_[reactor_idx];
+}
+
+Status Messenger::Init() {
+  RETURN_NOT_OK(tls_context_->Init());
+  for (Reactor* r : reactors_) {
+    RETURN_NOT_OK(r->Init());
+  }
+
+  return Status::OK();
+}
+
+Status Messenger::DumpRunningRpcs(const DumpRunningRpcsRequestPB& req,
+                                  DumpRunningRpcsResponsePB* resp) {
+  shared_lock<rw_spinlock> guard(lock_.get_lock());
+  for (Reactor* reactor : reactors_) {
+    RETURN_NOT_OK(reactor->DumpRunningRpcs(req, resp));
+  }
+  return Status::OK();
+}
+
+void Messenger::ScheduleOnReactor(const boost::function<void(const Status&)>& func,
+                                  MonoDelta when) {
+  DCHECK(!reactors_.empty());
+
+  // If we're already running on a reactor thread, reuse it.
+  Reactor* chosen = nullptr;
+  for (Reactor* r : reactors_) {
+    if (r->IsCurrentThread()) {
+      chosen = r;
+    }
+  }
+  if (chosen == nullptr) {
+    // Not running on a reactor thread, pick one at random.
+    chosen = reactors_[rand() % reactors_.size()];
+  }
+
+  DelayedTask* task = new DelayedTask(func, when);
+  chosen->ScheduleReactorTask(task);
+}
+
+const scoped_refptr<RpcService> Messenger::rpc_service(const string& service_name) const {
+  scoped_refptr<RpcService> service;
+  {
+    shared_lock<rw_spinlock> guard(lock_.get_lock());
+    if (!FindCopy(rpc_services_, service_name, &service)) {
+      return scoped_refptr<RpcService>(nullptr);
+    }
+  }
+  return service;
+}
+
+ThreadPool* Messenger::negotiation_pool(Connection::Direction dir) {
+  switch (dir) {
+    case Connection::CLIENT: return client_negotiation_pool_.get();
+    case Connection::SERVER: return server_negotiation_pool_.get();
+  }
+  DCHECK(false) << "Unknown Connection::Direction value: " << dir;
+  return nullptr;
+}
+
+} // namespace rpc
+} // namespace kudu
diff --git a/be/src/kudu/rpc/messenger.h b/be/src/kudu/rpc/messenger.h
new file mode 100644
index 0000000..64a804b
--- /dev/null
+++ b/be/src/kudu/rpc/messenger.h
@@ -0,0 +1,460 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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.
+#ifndef KUDU_RPC_MESSENGER_H
+#define KUDU_RPC_MESSENGER_H
+
+#include <cstdint>
+#include <memory>
+#include <mutex>
+#include <string>
+#include <unordered_map>
+#include <vector>
+
+#include <boost/optional/optional.hpp>
+#include <gtest/gtest_prod.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/rpc/connection.h"
+#include "kudu/security/security_flags.h"
+#include "kudu/security/token.pb.h"
+#include "kudu/util/locks.h"
+#include "kudu/util/metrics.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/status.h"
+
+namespace boost {
+template <typename Signature>
+class function;
+}
+
+namespace kudu {
+
+class Socket;
+class ThreadPool;
+
+namespace security {
+class TlsContext;
+class TokenVerifier;
+}
+
+namespace rpc {
+
+using security::RpcAuthentication;
+using security::RpcEncryption;
+
+class AcceptorPool;
+class DumpRunningRpcsRequestPB;
+class DumpRunningRpcsResponsePB;
+class InboundCall;
+class Messenger;
+class OutboundCall;
+class Reactor;
+class RpcService;
+class RpczStore;
+
+struct AcceptorPoolInfo {
+ public:
+  explicit AcceptorPoolInfo(Sockaddr bind_address)
+      : bind_address_(bind_address) {}
+
+  Sockaddr bind_address() const {
+    return bind_address_;
+  }
+
+ private:
+  Sockaddr bind_address_;
+};
+
+// Used to construct a Messenger.
+class MessengerBuilder {
+ public:
+  friend class Messenger;
+  friend class ReactorThread;
+
+  explicit MessengerBuilder(std::string name);
+
+  // Set the length of time we will keep a TCP connection will alive with no traffic.
+  MessengerBuilder &set_connection_keepalive_time(const MonoDelta &keepalive);
+
+  // Set the number of reactor threads that will be used for sending and
+  // receiving.
+  MessengerBuilder &set_num_reactors(int num_reactors);
+
+  // Set the minimum number of connection-negotiation threads that will be used
+  // to handle the blocking connection-negotiation step.
+  MessengerBuilder &set_min_negotiation_threads(int min_negotiation_threads);
+
+  // Set the maximum number of connection-negotiation threads that will be used
+  // to handle the blocking connection-negotiation step.
+  MessengerBuilder &set_max_negotiation_threads(int max_negotiation_threads);
+
+  // Set the granularity with which connections are checked for keepalive.
+  MessengerBuilder &set_coarse_timer_granularity(const MonoDelta &granularity);
+
+  // Set metric entity for use by RPC systems.
+  MessengerBuilder &set_metric_entity(const scoped_refptr<MetricEntity>& metric_entity);
+
+  // Set the time in milliseconds after which an idle connection from a client will be
+  // disconnected by the server.
+  MessengerBuilder &set_connection_keep_alive_time(int32_t time_in_ms);
+
+  // Set the timeout for negotiating an RPC connection.
+  MessengerBuilder &set_rpc_negotiation_timeout_ms(int64_t time_in_ms);
+
+  // Set the SASL protocol name that is used for the SASL negotiation.
+  MessengerBuilder &set_sasl_proto_name(const std::string& sasl_proto_name);
+
+  // Set the state of authentication required. If 'optional', authentication will be used when
+  // the remote end supports it. If 'required', connections which are not able to authenticate
+  // (because the remote end lacks support) are rejected.
+  MessengerBuilder &set_rpc_authentication(const std::string& rpc_authentication);
+
+  // Set the state of encryption required. If 'optional', encryption will be used when the
+  // remote end supports it. If 'required', connections which are not able to use encryption
+  // (because the remote end lacks support) are rejected. If 'disabled', encryption will not
+  // be used, and RPC authentication (--rpc_authentication) must also be disabled as well.
+  MessengerBuilder &set_rpc_encryption(const std::string& rpc_encryption);
+
+  // Set the cipher suite preferences to use for TLS-secured RPC connections. Uses the OpenSSL
+  // cipher preference list format. See man (1) ciphers for more information.
+  MessengerBuilder &set_rpc_tls_ciphers(const std::string& rpc_tls_ciphers);
+
+  // Set the minimum protocol version to allow when for securing RPC connections with TLS. May be
+  // one of 'TLSv1', 'TLSv1.1', or 'TLSv1.2'.
+  MessengerBuilder &set_rpc_tls_min_protocol(const std::string& rpc_tls_min_protocol);
+
+  // Set the TLS server certificate and private key files paths. If this is set in conjunction
+  // with enable_inbound_tls(), internal PKI will not be used for encrypted communication and
+  // external PKI will be used instead.
+  MessengerBuilder &set_epki_cert_key_files(
+      const std::string& cert, const std::string& private_key);
+
+  // Set the TLS Certificate Authority file path. Must always be set with set_epki_cert_key_files().
+  // If this is set in conjunction with enable_inbound_tls(), internal PKI will not be used for
+  // encrypted communication and external PKI will be used instead.
+  MessengerBuilder &set_epki_certificate_authority_file(const std::string& ca);
+
+  // Set a Unix command whose output returns the password used to decrypt the RPC server's private
+  // key file specified via set_epki_cert_key_files(). If the .PEM key file is not
+  // password-protected, this flag does not need to be set. Trailing whitespace will be trimmed
+  // before it is used to decrypt the private key.
+  MessengerBuilder &set_epki_private_password_key_cmd(const std::string& cmd);
+
+  // Set the path to the Kerberos Keytab file for this server.
+  MessengerBuilder &set_keytab_file(const std::string& keytab_file);
+
+  // Configure the messenger to enable TLS encryption on inbound connections.
+  MessengerBuilder& enable_inbound_tls();
+
+  // Configure the messenger to set the SO_REUSEPORT socket option.
+  MessengerBuilder& set_reuseport();
+
+  Status Build(std::shared_ptr<Messenger> *msgr);
+
+ private:
+  const std::string name_;
+  MonoDelta connection_keepalive_time_;
+  int num_reactors_;
+  int min_negotiation_threads_;
+  int max_negotiation_threads_;
+  MonoDelta coarse_timer_granularity_;
+  scoped_refptr<MetricEntity> metric_entity_;
+  int64_t rpc_negotiation_timeout_ms_;
+  std::string sasl_proto_name_;
+  std::string rpc_authentication_;
+  std::string rpc_encryption_;
+  std::string rpc_tls_ciphers_;
+  std::string rpc_tls_min_protocol_;
+  std::string rpc_certificate_file_;
+  std::string rpc_private_key_file_;
+  std::string rpc_ca_certificate_file_;
+  std::string rpc_private_key_password_cmd_;
+  std::string keytab_file_;
+  bool enable_inbound_tls_;
+  bool reuseport_;
+};
+
+// A Messenger is a container for the reactor threads which run event loops
+// for the RPC services. If the process is a server, a Messenger can also have
+// one or more attached AcceptorPools which accept RPC connections. In this case,
+// calls received over the connection are enqueued into the messenger's service_queue
+// for processing by a ServicePool.
+//
+// Users do not typically interact with the Messenger directly except to create
+// one as a singleton, and then make calls using Proxy objects.
+//
+// See rpc-test.cc and rpc-bench.cc for example usages.
+class Messenger {
+ public:
+  friend class MessengerBuilder;
+  friend class Proxy;
+  friend class Reactor;
+  friend class ReactorThread;
+  typedef std::vector<std::shared_ptr<AcceptorPool> > acceptor_vec_t;
+  typedef std::unordered_map<std::string, scoped_refptr<RpcService> > RpcServicesMap;
+
+  static const uint64_t UNKNOWN_CALL_ID = 0;
+
+  ~Messenger();
+
+  // Stops all communication and prevents further use. If called explicitly,
+  // also waits for outstanding tasks running on reactor threads to finish,
+  // which means it may  not be called from a reactor task.
+  //
+  // It's not required to call this -- dropping the shared_ptr provided
+  // from MessengerBuilder::Build will automatically call this method.
+  void Shutdown();
+
+  // Add a new acceptor pool listening to the given accept address.
+  // You can create any number of acceptor pools you want, including none.
+  //
+  // The created pool is returned in *pool. The Messenger also retains
+  // a reference to the pool, so the caller may safely drop this reference
+  // and the pool will remain live.
+  //
+  // NOTE: the returned pool is not initially started. You must call
+  // pool->Start(...) to begin accepting connections.
+  //
+  // If Kerberos is enabled, this also runs a pre-flight check that makes
+  // sure the environment is appropriately configured to authenticate
+  // clients via Kerberos. If not, this returns a RuntimeError.
+  Status AddAcceptorPool(const Sockaddr &accept_addr,
+                         std::shared_ptr<AcceptorPool>* pool);
+
+  // Register a new RpcService to handle inbound requests.
+  //
+  // Returns an error if a service with the same name is already registered.
+  Status RegisterService(const std::string& service_name,
+                         const scoped_refptr<RpcService>& service);
+
+  // Unregister an RpcService by name.
+  //
+  // Returns an error if no service with this name can be found.
+  Status UnregisterService(const std::string& service_name);
+
+  // Unregisters all RPC services.
+  void UnregisterAllServices();
+
+  // Queue a call for transmission. This will pick the appropriate reactor,
+  // and enqueue a task on that reactor to assign and send the call.
+  void QueueOutboundCall(const std::shared_ptr<OutboundCall> &call);
+
+  // Enqueue a call for processing on the server.
+  void QueueInboundCall(gscoped_ptr<InboundCall> call);
+
+  // Queue a cancellation for the given outbound call.
+  void QueueCancellation(const std::shared_ptr<OutboundCall> &call);
+
+  // Take ownership of the socket via Socket::Release
+  void RegisterInboundSocket(Socket *new_socket, const Sockaddr &remote);
+
+  // Dump the current RPCs into the given protobuf.
+  Status DumpRunningRpcs(const DumpRunningRpcsRequestPB& req,
+                         DumpRunningRpcsResponsePB* resp);
+
+  // Run 'func' on a reactor thread after 'when' time elapses.
+  //
+  // The status argument conveys whether 'func' was run correctly (i.e.
+  // after the elapsed time) or not.
+  void ScheduleOnReactor(const boost::function<void(const Status&)>& func,
+                         MonoDelta when);
+
+  const security::TlsContext& tls_context() const { return *tls_context_; }
+  security::TlsContext* mutable_tls_context() { return tls_context_.get(); }
+
+  const security::TokenVerifier& token_verifier() const { return *token_verifier_; }
+  security::TokenVerifier* mutable_token_verifier() { return token_verifier_.get(); }
+  std::shared_ptr<security::TokenVerifier> shared_token_verifier() const {
+    return token_verifier_;
+  }
+
+  boost::optional<security::SignedTokenPB> authn_token() const {
+    std::lock_guard<simple_spinlock> l(authn_token_lock_);
+    return authn_token_;
+  }
+  void set_authn_token(const security::SignedTokenPB& token) {
+    std::lock_guard<simple_spinlock> l(authn_token_lock_);
+    authn_token_ = token;
+  }
+
+  RpcAuthentication authentication() const { return authentication_; }
+  RpcEncryption encryption() const { return encryption_; }
+
+  ThreadPool* negotiation_pool(Connection::Direction dir);
+
+  RpczStore* rpcz_store() { return rpcz_store_.get(); }
+
+  int num_reactors() const { return reactors_.size(); }
+
+  const std::string& name() const {
+    return name_;
+  }
+
+  bool closing() const {
+    shared_lock<rw_spinlock> l(lock_.get_lock());
+    return closing_;
+  }
+
+  scoped_refptr<MetricEntity> metric_entity() const { return metric_entity_; }
+
+  const int64_t rpc_negotiation_timeout_ms() const { return rpc_negotiation_timeout_ms_; }
+
+  const std::string& sasl_proto_name() const {
+    return sasl_proto_name_;
+  }
+
+  const std::string& keytab_file() const { return keytab_file_; }
+
+  const scoped_refptr<RpcService> rpc_service(const std::string& service_name) const;
+
+ private:
+  FRIEND_TEST(TestRpc, TestConnectionKeepalive);
+  FRIEND_TEST(TestRpc, TestConnectionAlwaysKeepalive);
+  FRIEND_TEST(TestRpc, TestClientConnectionsMetrics);
+  FRIEND_TEST(TestRpc, TestCredentialsPolicy);
+  FRIEND_TEST(TestRpc, TestReopenOutboundConnections);
+
+  explicit Messenger(const MessengerBuilder &bld);
+
+  Reactor* RemoteToReactor(const Sockaddr &remote);
+  Status Init();
+  void RunTimeoutThread();
+  void UpdateCurTime();
+
+  // Shuts down the messenger.
+  //
+  // Depending on 'mode', may or may not wait on any outstanding reactor tasks.
+  enum class ShutdownMode {
+    SYNC,
+    ASYNC,
+  };
+  void ShutdownInternal(ShutdownMode mode);
+
+  // Called by external-facing shared_ptr when the user no longer holds
+  // any references. See 'retain_self_' for more info.
+  void AllExternalReferencesDropped();
+
+  const std::string name_;
+
+  // Protects closing_, acceptor_pools_, rpc_services_.
+  mutable percpu_rwlock lock_;
+
+  bool closing_;
+
+  // Whether to require authentication and encryption on the connections managed
+  // by this messenger.
+  // TODO(KUDU-1928): scope these to individual proxies, so that messengers can be
+  // reused by different clients.
+  RpcAuthentication authentication_;
+  RpcEncryption encryption_;
+
+  // Pools which are listening on behalf of this messenger.
+  // Note that the user may have called Shutdown() on one of these
+  // pools, so even though we retain the reference, it may no longer
+  // be listening.
+  acceptor_vec_t acceptor_pools_;
+
+  // RPC services that handle inbound requests.
+  RpcServicesMap rpc_services_;
+
+  std::vector<Reactor*> reactors_;
+
+  // Separate client and server negotiation pools to avoid possibility of distributed
+  // deadlock. See KUDU-2041.
+  gscoped_ptr<ThreadPool> client_negotiation_pool_;
+  gscoped_ptr<ThreadPool> server_negotiation_pool_;
+
+  std::unique_ptr<security::TlsContext> tls_context_;
+
+  // A TokenVerifier, which can verify client provided authentication tokens.
+  std::shared_ptr<security::TokenVerifier> token_verifier_;
+
+  // An optional token, which can be used to authenticate to a server.
+  mutable simple_spinlock authn_token_lock_;
+  boost::optional<security::SignedTokenPB> authn_token_;
+
+  std::unique_ptr<RpczStore> rpcz_store_;
+
+  scoped_refptr<MetricEntity> metric_entity_;
+
+  // Timeout in milliseconds after which an incomplete connection negotiation will timeout.
+  const int64_t rpc_negotiation_timeout_ms_;
+
+  // The SASL protocol name that is used for the SASL negotiation.
+  const std::string sasl_proto_name_;
+
+  // Path to the Kerberos Keytab file for this server.
+  const std::string keytab_file_;
+
+  // Whether to set SO_REUSEPORT on the listening sockets.
+  bool reuseport_;
+
+  // The ownership of the Messenger object is somewhat subtle. The pointer graph
+  // looks like this:
+  //
+  //    [User Code ]             |      [ Internal code ]
+  //                             |
+  //     shared_ptr[1]           |
+  //         |                   |
+  //         v
+  //      Messenger    <------------ shared_ptr[2] --- Reactor
+  //       ^    |      ------------- bare pointer  --> Reactor
+  //        \__/
+  //     shared_ptr[2]
+  //     (retain_self_)
+  //
+  // shared_ptr[1] instances use Messenger::AllExternalReferencesDropped()
+  //   as a deleter.
+  // shared_ptr[2] are "traditional" shared_ptrs which call 'delete' on the
+  //   object.
+  //
+  // The teardown sequence is as follows:
+  // Option 1): User calls "Shutdown()" explicitly:
+  //  - Messenger::Shutdown tells Reactors to shut down.
+  //  - When each reactor thread finishes, it drops its shared_ptr[2].
+  //  - the Messenger::retain_self instance remains, keeping the Messenger
+  //    alive.
+  //  - Before returning, Messenger::Shutdown waits for Reactors to shut down.
+  //  - The user eventually drops its shared_ptr[1], which calls
+  //    Messenger::AllExternalReferencesDropped. This drops retain_self_
+  //    and results in object destruction.
+  // Option 2): User drops all of its shared_ptr[1] references
+  //  - Though the Reactors still reference the Messenger, AllExternalReferencesDropped
+  //    will get called, which triggers Messenger::Shutdown.
+  //  - AllExternalReferencesDropped drops retain_self_, so the only remaining
+  //    references are from Reactor threads. But the reactor threads are shutting down.
+  //  - When the last Reactor thread dies, there will be no more shared_ptr[1] references
+  //    and the Messenger will be destroyed.
+  //
+  // The main goal of all of this confusion is that when using option 2, the
+  // reactor threads need to be able to shut down asynchronously, and we need
+  // to keep the Messenger alive until they do so. If normal shared_ptrs were
+  // handed out to users, the Messenger destructor may be forced to Join() the
+  // reactor threads, which deadlocks if the user destructs the Messenger from
+  // within a Reactor thread itself.
+  std::shared_ptr<Messenger> retain_self_;
+
+  DISALLOW_COPY_AND_ASSIGN(Messenger);
+};
+
+} // namespace rpc
+} // namespace kudu
+
+#endif
diff --git a/be/src/kudu/rpc/mt-rpc-test.cc b/be/src/kudu/rpc/mt-rpc-test.cc
new file mode 100644
index 0000000..7427850
--- /dev/null
+++ b/be/src/kudu/rpc/mt-rpc-test.cc
@@ -0,0 +1,318 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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.
+
+#include "kudu/rpc/rpc-test-base.h"
+
+#include <cstddef>
+#include <memory>
+#include <ostream>
+#include <string>
+#include <type_traits>
+#include <utility>
+#include <vector>
+
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/rpc/acceptor_pool.h"
+#include "kudu/rpc/messenger.h"
+#include "kudu/rpc/proxy.h"
+#include "kudu/rpc/rpc_service.h"
+#include "kudu/rpc/service_if.h"
+#include "kudu/rpc/service_pool.h"
+#include "kudu/util/countdown_latch.h"
+#include "kudu/util/metrics.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/net/socket.h"
+#include "kudu/util/status.h"
+#include "kudu/util/stopwatch.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/thread.h"
+
+
+METRIC_DECLARE_counter(rpc_connections_accepted);
+METRIC_DECLARE_counter(rpcs_queue_overflow);
+
+using std::string;
+using std::shared_ptr;
+using std::vector;
+using strings::Substitute;
+
+namespace kudu {
+namespace rpc {
+
+class MultiThreadedRpcTest : public RpcTestBase {
+ public:
+  // Make a single RPC call.
+  void SingleCall(Sockaddr server_addr, const char* method_name,
+                  Status* result, CountDownLatch* latch) {
+    LOG(INFO) << "Connecting to " << server_addr.ToString();
+    shared_ptr<Messenger> client_messenger;
+    CHECK_OK(CreateMessenger("ClientSC", &client_messenger));
+    Proxy p(client_messenger, server_addr, server_addr.host(),
+            GenericCalculatorService::static_service_name());
+    *result = DoTestSyncCall(p, method_name);
+    latch->CountDown();
+  }
+
+  // Make RPC calls until we see a failure.
+  void HammerServer(Sockaddr server_addr, const char* method_name,
+                    Status* last_result) {
+    shared_ptr<Messenger> client_messenger;
+    CHECK_OK(CreateMessenger("ClientHS", &client_messenger));
+    HammerServerWithMessenger(server_addr, method_name, last_result, client_messenger);
+  }
+
+  void HammerServerWithMessenger(
+      Sockaddr server_addr, const char* method_name, Status* last_result,
+      const shared_ptr<Messenger>& messenger) {
+    LOG(INFO) << "Connecting to " << server_addr.ToString();
+    Proxy p(messenger, server_addr, server_addr.host(),
+            GenericCalculatorService::static_service_name());
+
+    int i = 0;
+    while (true) {
+      i++;
+      Status s = DoTestSyncCall(p, method_name);
+      if (!s.ok()) {
+        // Return on first failure.
+        LOG(INFO) << "Call failed. Shutting down client thread. Ran " << i << " calls: "
+            << s.ToString();
+        *last_result = s;
+        return;
+      }
+    }
+  }
+};
+
+static void AssertShutdown(kudu::Thread* thread, const Status* status) {
+  ASSERT_OK(ThreadJoiner(thread).warn_every_ms(500).Join());
+  string msg = status->ToString();
+  ASSERT_TRUE(msg.find("Service unavailable") != string::npos ||
+              msg.find("Network error") != string::npos)
+              << "Status is actually: " << msg;
+}
+
+// Test making several concurrent RPC calls while shutting down.
+// Simply verify that we don't hit any CHECK errors.
+TEST_F(MultiThreadedRpcTest, TestShutdownDuringService) {
+  // Set up server.
+  Sockaddr server_addr;
+  ASSERT_OK(StartTestServer(&server_addr));
+
+  const int kNumThreads = 4;
+  scoped_refptr<kudu::Thread> threads[kNumThreads];
+  Status statuses[kNumThreads];
+  for (int i = 0; i < kNumThreads; i++) {
+    ASSERT_OK(kudu::Thread::Create("test", strings::Substitute("t$0", i),
+      &MultiThreadedRpcTest::HammerServer, this, server_addr,
+      GenericCalculatorService::kAddMethodName, &statuses[i], &threads[i]));
+  }
+
+  SleepFor(MonoDelta::FromMilliseconds(50));
+
+  // Shut down server.
+  ASSERT_OK(server_messenger_->UnregisterService(service_name_));
+  service_pool_->Shutdown();
+  server_messenger_->Shutdown();
+
+  for (int i = 0; i < kNumThreads; i++) {
+    AssertShutdown(threads[i].get(), &statuses[i]);
+  }
+}
+
+// Test shutting down the client messenger exactly as a thread is about to start
+// a new connection. This is a regression test for KUDU-104.
+TEST_F(MultiThreadedRpcTest, TestShutdownClientWhileCallsPending) {
+  // Set up server.
+  Sockaddr server_addr;
+  ASSERT_OK(StartTestServer(&server_addr));
+
+  shared_ptr<Messenger> client_messenger;
+  ASSERT_OK(CreateMessenger("Client", &client_messenger));
+
+  scoped_refptr<kudu::Thread> thread;
+  Status status;
+  ASSERT_OK(kudu::Thread::Create("test", "test",
+      &MultiThreadedRpcTest::HammerServerWithMessenger, this, server_addr,
+      GenericCalculatorService::kAddMethodName, &status, client_messenger, &thread));
+
+  // Shut down the messenger after a very brief sleep. This often will race so that the
+  // call gets submitted to the messenger before shutdown, but the negotiation won't have
+  // started yet. In a debug build this fails about half the time without the bug fix.
+  // See KUDU-104.
+  SleepFor(MonoDelta::FromMicroseconds(10));
+  client_messenger->Shutdown();
+  client_messenger.reset();
+
+  ASSERT_OK(ThreadJoiner(thread.get()).warn_every_ms(500).Join());
+  ASSERT_TRUE(status.IsAborted() ||
+              status.IsServiceUnavailable());
+  string msg = status.ToString();
+  SCOPED_TRACE(msg);
+  ASSERT_TRUE(msg.find("Client RPC Messenger shutting down") != string::npos ||
+              msg.find("reactor is shutting down") != string::npos ||
+              msg.find("Unable to start connection negotiation thread") != string::npos)
+              << "Status is actually: " << msg;
+}
+
+// This bogus service pool leaves the service queue full.
+class BogusServicePool : public ServicePool {
+ public:
+  BogusServicePool(gscoped_ptr<ServiceIf> service,
+                   const scoped_refptr<MetricEntity>& metric_entity,
+                   size_t service_queue_length)
+    : ServicePool(std::move(service), metric_entity, service_queue_length) {
+  }
+  virtual Status Init(int num_threads) OVERRIDE {
+    // Do nothing
+    return Status::OK();
+  }
+};
+
+void IncrementBackpressureOrShutdown(const Status* status, int* backpressure, int* shutdown) {
+  string msg = status->ToString();
+  if (msg.find("service queue is full") != string::npos) {
+    ++(*backpressure);
+  } else if (msg.find("shutting down") != string::npos) {
+    ++(*shutdown);
+  } else if (msg.find("got EOF from remote") != string::npos) {
+    ++(*shutdown);
+  } else {
+    FAIL() << "Unexpected status message: " << msg;
+  }
+}
+
+// Test that we get a Service Unavailable error when we max out the incoming RPC service queue.
+TEST_F(MultiThreadedRpcTest, TestBlowOutServiceQueue) {
+  const size_t kMaxConcurrency = 2;
+
+  MessengerBuilder bld("messenger1");
+  bld.set_num_reactors(kMaxConcurrency);
+  bld.set_metric_entity(metric_entity_);
+  CHECK_OK(bld.Build(&server_messenger_));
+
+  shared_ptr<AcceptorPool> pool;
+  ASSERT_OK(server_messenger_->AddAcceptorPool(Sockaddr(), &pool));
+  ASSERT_OK(pool->Start(kMaxConcurrency));
+  Sockaddr server_addr = pool->bind_address();
+
+  gscoped_ptr<ServiceIf> service(new GenericCalculatorService());
+  service_name_ = service->service_name();
+  service_pool_ = new BogusServicePool(std::move(service),
+                                      server_messenger_->metric_entity(),
+                                      kMaxConcurrency);
+  ASSERT_OK(service_pool_->Init(n_worker_threads_));
+  server_messenger_->RegisterService(service_name_, service_pool_);
+
+  scoped_refptr<kudu::Thread> threads[3];
+  Status status[3];
+  CountDownLatch latch(1);
+  for (int i = 0; i < 3; i++) {
+    ASSERT_OK(kudu::Thread::Create("test", strings::Substitute("t$0", i),
+      &MultiThreadedRpcTest::SingleCall, this, server_addr,
+      GenericCalculatorService::kAddMethodName, &status[i], &latch, &threads[i]));
+  }
+
+  // One should immediately fail due to backpressure. The latch is only initialized
+  // to wait for the first of three threads to finish.
+  latch.Wait();
+
+  // The rest would time out after 10 sec, but we help them along.
+  ASSERT_OK(server_messenger_->UnregisterService(service_name_));
+  service_pool_->Shutdown();
+  server_messenger_->Shutdown();
+
+  for (const auto& thread : threads) {
+    ASSERT_OK(ThreadJoiner(thread.get()).warn_every_ms(500).Join());
+  }
+
+  // Verify that one error was due to backpressure.
+  int errors_backpressure = 0;
+  int errors_shutdown = 0;
+
+  for (const auto& s : status) {
+    IncrementBackpressureOrShutdown(&s, &errors_backpressure, &errors_shutdown);
+  }
+
+  ASSERT_EQ(1, errors_backpressure);
+  ASSERT_EQ(2, errors_shutdown);
+
+  // Check that RPC queue overflow metric is 1
+  Counter *rpcs_queue_overflow =
+    METRIC_rpcs_queue_overflow.Instantiate(server_messenger_->metric_entity()).get();
+  ASSERT_EQ(1, rpcs_queue_overflow->value());
+}
+
+static void HammerServerWithTCPConns(const Sockaddr& addr) {
+  while (true) {
+    Socket socket;
+    CHECK_OK(socket.Init(0));
+    Status s;
+    LOG_SLOW_EXECUTION(INFO, 100, "Connect took long") {
+      s = socket.Connect(addr);
+    }
+    if (!s.ok()) {
+      CHECK(s.IsNetworkError()) << "Unexpected error: " << s.ToString();
+      return;
+    }
+    CHECK_OK(socket.Close());
+  }
+}
+
+// Regression test for KUDU-128.
+// Test that shuts down the server while new TCP connections are incoming.
+TEST_F(MultiThreadedRpcTest, TestShutdownWithIncomingConnections) {
+  // Set up server.
+  Sockaddr server_addr;
+  ASSERT_OK(StartTestServer(&server_addr));
+
+  // Start a number of threads which just hammer the server with TCP connections.
+  vector<scoped_refptr<kudu::Thread> > threads;
+  for (int i = 0; i < 8; i++) {
+    scoped_refptr<kudu::Thread> new_thread;
+    CHECK_OK(kudu::Thread::Create("test", strings::Substitute("t$0", i),
+        &HammerServerWithTCPConns, server_addr, &new_thread));
+    threads.push_back(new_thread);
+  }
+
+  // Sleep until the server has started to actually accept some connections from the
+  // test threads.
+  scoped_refptr<Counter> conns_accepted =
+    METRIC_rpc_connections_accepted.Instantiate(server_messenger_->metric_entity());
+  while (conns_accepted->value() == 0) {
+    SleepFor(MonoDelta::FromMicroseconds(100));
+  }
+
+  // Shutdown while there are still new connections appearing.
+  ASSERT_OK(server_messenger_->UnregisterService(service_name_));
+  service_pool_->Shutdown();
+  server_messenger_->Shutdown();
+
+  for (scoped_refptr<kudu::Thread>& t : threads) {
+    ASSERT_OK(ThreadJoiner(t.get()).warn_every_ms(500).Join());
+  }
+}
+
+} // namespace rpc
+} // namespace kudu
+
diff --git a/be/src/kudu/rpc/negotiation-test.cc b/be/src/kudu/rpc/negotiation-test.cc
new file mode 100644
index 0000000..976f590
--- /dev/null
+++ b/be/src/kudu/rpc/negotiation-test.cc
@@ -0,0 +1,1346 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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.
+
+#include "kudu/rpc/rpc-test-base.h"
+
+#include <sys/stat.h>
+#include <unistd.h>
+
+#include <cstdio>
+#include <cstdlib>
+#include <functional>
+#include <memory>
+#include <ostream>
+#include <string>
+#include <thread>
+#include <utility>
+#include <vector>
+
+#include <boost/optional/optional.hpp>
+#include <gflags/gflags.h>
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+#include <sasl/sasl.h>
+
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/strings/join.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/walltime.h"
+#include "kudu/rpc/client_negotiation.h"
+#include "kudu/rpc/messenger.h"
+#include "kudu/rpc/negotiation.h"
+#include "kudu/rpc/remote_user.h"
+#include "kudu/rpc/sasl_common.h"
+#include "kudu/rpc/server_negotiation.h"
+#include "kudu/security/cert.h"
+#include "kudu/security/crypto.h"
+#include "kudu/security/security-test-util.h"
+#include "kudu/security/security_flags.h"
+#include "kudu/security/test/mini_kdc.h"
+#include "kudu/security/tls_context.h"
+#include "kudu/security/tls_socket.h"
+#include "kudu/security/token.pb.h"
+#include "kudu/security/token_signer.h"
+#include "kudu/security/token_signing_key.h"
+#include "kudu/security/token_verifier.h"
+#include "kudu/util/env.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/net/sockaddr.h"
+#include "kudu/util/net/socket.h"
+#include "kudu/util/status.h"
+#include "kudu/util/subprocess.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+#include "kudu/util/trace.h"
+#include "kudu/util/user.h"
+
+// HACK: MIT Kerberos doesn't have any way of determining its version number,
+// but the error messages in krb5-1.10 and earlier are broken due to
+// a bug: http://krbdev.mit.edu/rt/Ticket/Display.html?id=6973
+//
+// Since we don't have any way to explicitly figure out the version, we just
+// look for this random macro which was added in 1.11 (the same version in which
+// the above bug was fixed).
+#ifndef KRB5_RESPONDER_QUESTION_PASSWORD
+#define KRB5_VERSION_LE_1_10
+#endif
+
+DEFINE_bool(is_test_child, false,
+            "Used by tests which require clean processes. "
+            "See TestDisableInit.");
+DECLARE_bool(rpc_encrypt_loopback_connections);
+DECLARE_bool(rpc_trace_negotiation);
+
+using std::string;
+using std::thread;
+using std::unique_ptr;
+using std::vector;
+
+using kudu::security::Cert;
+using kudu::security::PkiConfig;
+using kudu::security::PrivateKey;
+using kudu::security::SignedTokenPB;
+using kudu::security::TlsContext;
+using kudu::security::TokenSigner;
+using kudu::security::TokenSigningPrivateKey;
+using kudu::security::TokenVerifier;
+
+namespace kudu {
+namespace rpc {
+
+// The negotiation configuration for a client or server endpoint.
+struct EndpointConfig {
+  // The PKI configuration.
+  PkiConfig pki;
+  // The supported SASL mechanisms.
+  vector<SaslMechanism::Type> sasl_mechs;
+  // For the client, whether the client has the token.
+  // For the server, whether the server has the TSK.
+  bool token;
+  RpcEncryption encryption;
+};
+std::ostream& operator<<(std::ostream& o, EndpointConfig config) {
+  auto bool_string = [] (bool b) { return b ? "true" : "false"; };
+  o << "{pki: " << config.pki
+    << ", sasl-mechs: [" << JoinMapped(config.sasl_mechs, SaslMechanism::name_of, ", ")
+    << "], token: " << bool_string(config.token)
+    << ", encryption: ";
+
+  switch (config.encryption) {
+    case RpcEncryption::DISABLED: o << "DISABLED"; break;
+    case RpcEncryption::OPTIONAL: o << "OPTIONAL"; break;
+    case RpcEncryption::REQUIRED: o << "REQUIRED"; break;
+  }
+
+  o << "}";
+  return o;
+}
+
+// A description of a negotiation sequence, including client and server
+// configuration, as well as expected results.
+struct NegotiationDescriptor {
+  EndpointConfig client;
+  EndpointConfig server;
+
+  bool use_test_socket;
+
+  bool rpc_encrypt_loopback;
+
+  // The expected client status from negotiating.
+  Status client_status;
+  // The expected server status from negotiating.
+  Status server_status;
+
+  // The expected negotiated authentication type.
+  AuthenticationType negotiated_authn;
+
+  // The expected SASL mechanism, if SASL authentication is negotiated.
+  SaslMechanism::Type negotiated_mech;
+
+  // Whether the negotiation is expected to perform a TLS handshake.
+  bool tls_negotiated;
+};
+std::ostream& operator<<(std::ostream& o, NegotiationDescriptor c) {
+  auto bool_string = [] (bool b) { return b ? "true" : "false"; };
+  o << "{client: " << c.client
+    << ", server: " << c.server
+    << "}, rpc-encrypt-loopback: " << bool_string(c.rpc_encrypt_loopback);
+  return o;
+}
+
+class NegotiationTestSocket : public Socket {
+ public:
+  // Return an arbitrary public IP
+  Status GetPeerAddress(Sockaddr *cur_addr) const override {
+    return cur_addr->ParseString("8.8.8.8:12345", 0);
+  }
+};
+
+class TestNegotiation : public RpcTestBase,
+                        public ::testing::WithParamInterface<NegotiationDescriptor> {
+ public:
+  void SetUp() override {
+    RpcTestBase::SetUp();
+    ASSERT_OK(SaslInit());
+  }
+};
+
+TEST_P(TestNegotiation, TestNegotiation) {
+  NegotiationDescriptor desc = GetParam();
+
+  // Generate a trusted root certificate.
+  PrivateKey ca_key;
+  Cert ca_cert;
+  ASSERT_OK(GenerateSelfSignedCAForTests(&ca_key, &ca_cert));
+
+  // Create and configure a TLS context for each endpoint.
+  TlsContext client_tls_context;
+  TlsContext server_tls_context;
+  ASSERT_OK(client_tls_context.Init());
+  ASSERT_OK(server_tls_context.Init());
+  ASSERT_OK(ConfigureTlsContext(desc.client.pki, ca_cert, ca_key, &client_tls_context));
+  ASSERT_OK(ConfigureTlsContext(desc.server.pki, ca_cert, ca_key, &server_tls_context));
+
+  FLAGS_rpc_encrypt_loopback_connections = desc.rpc_encrypt_loopback;
+
+  // Generate an optional client token and server token verifier.
+  TokenSigner token_signer(60, 20, std::make_shared<TokenVerifier>());
+  {
+    unique_ptr<TokenSigningPrivateKey> key;
+    ASSERT_OK(token_signer.CheckNeedKey(&key));
+    // No keys are available yet, so should be able to add.
+    ASSERT_NE(nullptr, key.get());
+    ASSERT_OK(token_signer.AddKey(std::move(key)));
+  }
+  TokenVerifier token_verifier;
+  boost::optional<SignedTokenPB> authn_token;
+  if (desc.client.token) {
+    authn_token = SignedTokenPB();
+    security::TokenPB token;
+    token.set_expire_unix_epoch_seconds(WallTime_Now() + 60);
+    token.mutable_authn()->set_username("client-token");
+    ASSERT_TRUE(token.SerializeToString(authn_token->mutable_token_data()));
+    ASSERT_OK(token_signer.SignToken(&*authn_token));
+  }
+  if (desc.server.token) {
+    ASSERT_OK(token_verifier.ImportKeys(token_signer.verifier().ExportKeys()));
+  }
+
+  // Create the listening socket, client socket, and server socket.
+  Socket listening_socket;
+  ASSERT_OK(listening_socket.Init(0));
+  ASSERT_OK(listening_socket.BindAndListen(Sockaddr(), 1));
+  Sockaddr server_addr;
+  ASSERT_OK(listening_socket.GetSocketAddress(&server_addr));
+
+  unique_ptr<Socket> client_socket(new Socket());
+  ASSERT_OK(client_socket->Init(0));
+  client_socket->Connect(server_addr);
+
+  unique_ptr<Socket> server_socket(desc.use_test_socket ?
+                                   new NegotiationTestSocket() :
+                                   new Socket());
+
+  Sockaddr client_addr;
+  CHECK_OK(listening_socket.Accept(server_socket.get(), &client_addr, 0));
+
+  // Create and configure the client and server negotiation instances.
+  ClientNegotiation client_negotiation(std::move(client_socket),
+                                       &client_tls_context,
+                                       authn_token,
+                                       desc.client.encryption,
+                                       "kudu");
+  ServerNegotiation server_negotiation(std::move(server_socket),
+                                       &server_tls_context,
+                                       &token_verifier,
+                                       desc.server.encryption,
+                                       "kudu");
+
+  // Set client and server SASL mechanisms.
+  MiniKdc kdc;
+  bool kdc_started = false;
+  auto start_kdc_once = [&] () {
+    if (!kdc_started) {
+      kdc_started = true;
+      RETURN_NOT_OK(kdc.Start());
+    }
+    return Status::OK();
+  };
+  for (auto mech : desc.client.sasl_mechs) {
+    switch (mech) {
+      case SaslMechanism::INVALID: break;
+      case SaslMechanism::PLAIN:
+        ASSERT_OK(client_negotiation.EnablePlain("client-plain", "client-password"));
+        break;
+      case SaslMechanism::GSSAPI:
+        ASSERT_OK(start_kdc_once());
+        ASSERT_OK(kdc.CreateUserPrincipal("client-gssapi"));
+        ASSERT_OK(kdc.Kinit("client-gssapi"));
+        ASSERT_OK(kdc.SetKrb5Environment());
+        client_negotiation.set_server_fqdn("127.0.0.1");
+        ASSERT_OK(client_negotiation.EnableGSSAPI());
+        break;
+    }
+  }
+  for (auto mech : desc.server.sasl_mechs) {
+    switch (mech) {
+      case SaslMechanism::INVALID: break;
+      case SaslMechanism::PLAIN:
+        ASSERT_OK(server_negotiation.EnablePlain());
+        break;
+      case SaslMechanism::GSSAPI:
+        ASSERT_OK(start_kdc_once());
+        // Create the server principal and keytab.
+        string kt_path;
+        ASSERT_OK(kdc.CreateServiceKeytab("kudu/127.0.0.1", &kt_path));
+        CHECK_ERR(setenv("KRB5_KTNAME", kt_path.c_str(), 1 /*replace*/));
+        server_negotiation.set_server_fqdn("127.0.0.1");
+        ASSERT_OK(server_negotiation.EnableGSSAPI());
+        break;
+    }
+  }
+
+  // Run the client/server negotiation. Because negotiation is blocking, it
+  // has to be done on separate threads.
+  Status client_status;
+  Status server_status;
+  thread client_thread([&] () {
+      scoped_refptr<Trace> t(new Trace());
+      ADOPT_TRACE(t.get());
+      client_status = client_negotiation.Negotiate();
+      // Close the socket so that the server will not block forever on error.
+      client_negotiation.socket()->Close();
+
+      if (FLAGS_rpc_trace_negotiation || !client_status.ok()) {
+        string msg = Trace::CurrentTrace()->DumpToString();
+        if (!client_status.ok()) {
+          LOG(WARNING) << "Failed client RPC negotiation. Client trace:\n" << msg;
+        } else {
+          LOG(INFO) << "RPC negotiation tracing enabled. Client trace:\n" << msg;
+        }
+      }
+  });
+  thread server_thread([&] () {
+      scoped_refptr<Trace> t(new Trace());
+      ADOPT_TRACE(t.get());
+      server_status = server_negotiation.Negotiate();
+      // Close the socket so that the client will not block forever on error.
+      server_negotiation.socket()->Close();
+
+      if (FLAGS_rpc_trace_negotiation || !server_status.ok()) {
+        string msg = Trace::CurrentTrace()->DumpToString();
+        if (!server_status.ok()) {
+          LOG(WARNING) << "Failed server RPC negotiation. Server trace:\n" << msg;
+        } else {
+          LOG(INFO) << "RPC negotiation tracing enabled. Server trace:\n" << msg;
+        }
+      }
+  });
+  client_thread.join();
+  server_thread.join();
+
+  // Check the negotiation outcome against the expected outcome.
+  EXPECT_EQ(desc.client_status.CodeAsString(), client_status.CodeAsString());
+  EXPECT_EQ(desc.server_status.CodeAsString(), server_status.CodeAsString());
+  ASSERT_STR_MATCHES(client_status.ToString(), desc.client_status.ToString());
+  ASSERT_STR_MATCHES(server_status.ToString(), desc.server_status.ToString());
+
+  if (client_status.ok()) {
+    EXPECT_TRUE(server_status.ok());
+
+    // Make sure the negotiations agree with the expected values.
+    EXPECT_EQ(desc.negotiated_authn, client_negotiation.negotiated_authn());
+    EXPECT_EQ(desc.negotiated_mech, client_negotiation.negotiated_mechanism());
+    EXPECT_EQ(desc.negotiated_authn, server_negotiation.negotiated_authn());
+    EXPECT_EQ(desc.negotiated_mech, server_negotiation.negotiated_mechanism());
+    EXPECT_EQ(desc.tls_negotiated, server_negotiation.tls_negotiated());
+    EXPECT_EQ(desc.tls_negotiated, server_negotiation.tls_negotiated());
+
+    bool client_tls_socket = dynamic_cast<security::TlsSocket*>(client_negotiation.socket());
+    bool server_tls_socket = dynamic_cast<security::TlsSocket*>(server_negotiation.socket());
+    EXPECT_EQ(desc.rpc_encrypt_loopback, client_tls_socket);
+    EXPECT_EQ(desc.rpc_encrypt_loopback, server_tls_socket);
+
+    // Check that the expected user subject is authenticated.
+    RemoteUser remote_user = server_negotiation.take_authenticated_user();
+    switch (server_negotiation.negotiated_authn()) {
+      case AuthenticationType::SASL:
+        switch (server_negotiation.negotiated_mechanism()) {
+          case SaslMechanism::PLAIN:
+            EXPECT_EQ("client-plain", remote_user.username());
+            break;
+          case SaslMechanism::GSSAPI:
+            EXPECT_EQ("client-gssapi", remote_user.username());
+            EXPECT_EQ("client-gssapi@KRBTEST.COM", remote_user.principal().value_or(""));
+            break;
+          case SaslMechanism::INVALID: LOG(FATAL) << "invalid mechanism negotiated";
+        }
+        break;
+      case AuthenticationType::CERTIFICATE: {
+        // We expect the cert to be using the local username, because it hasn't
+        // logged in from any Keytab.
+        string expected;
+        CHECK_OK(GetLoggedInUser(&expected));
+        EXPECT_EQ(expected, remote_user.username());
+        EXPECT_FALSE(remote_user.principal());
+        break;
+      }
+      case AuthenticationType::TOKEN:
+        EXPECT_EQ("client-token", remote_user.username());
+        break;
+      case AuthenticationType::INVALID: LOG(FATAL) << "invalid authentication negotiated";
+    }
+  }
+}
+
+INSTANTIATE_TEST_CASE_P(NegotiationCombinations,
+                        TestNegotiation,
+                        ::testing::Values(
+
+        // client: no authn/mechs
+        // server: no authn/mechs
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::NONE,
+            {},
+            false,
+            RpcEncryption::OPTIONAL,
+          },
+          EndpointConfig {
+            PkiConfig::NONE,
+            {},
+            false,
+            RpcEncryption::OPTIONAL,
+          },
+          false,
+          false,
+          Status::NotAuthorized(".*client is not configured with an authentication type"),
+          Status::NetworkError(""),
+          AuthenticationType::INVALID,
+          SaslMechanism::INVALID,
+          false,
+        },
+
+        // client: PLAIN
+        // server: no authn/mechs
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::NONE,
+            { SaslMechanism::PLAIN },
+            false,
+            RpcEncryption::OPTIONAL,
+          },
+          EndpointConfig {
+            PkiConfig::NONE,
+            {},
+            false,
+            RpcEncryption::OPTIONAL,
+          },
+          false,
+          false,
+          Status::NotAuthorized(".* server mechanism list is empty"),
+          Status::NotAuthorized(".* server mechanism list is empty"),
+          AuthenticationType::INVALID,
+          SaslMechanism::INVALID,
+          false,
+        },
+
+        // client: PLAIN
+        // server: PLAIN
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::NONE,
+            { SaslMechanism::PLAIN },
+            false,
+            RpcEncryption::OPTIONAL
+          },
+          EndpointConfig {
+            PkiConfig::NONE,
+            { SaslMechanism::PLAIN },
+            false,
+            RpcEncryption::DISABLED,
+          },
+          false,
+          false,
+          Status::OK(),
+          Status::OK(),
+          AuthenticationType::SASL,
+          SaslMechanism::PLAIN,
+          false,
+        },
+
+        // client: GSSAPI
+        // server: GSSAPI
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::NONE,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::OPTIONAL,
+          },
+          EndpointConfig {
+            PkiConfig::NONE,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::DISABLED,
+          },
+          false,
+          false,
+          Status::OK(),
+          Status::OK(),
+          AuthenticationType::SASL,
+          SaslMechanism::GSSAPI,
+          false,
+        },
+
+        // client: GSSAPI, PLAIN
+        // server: GSSAPI, PLAIN
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::NONE,
+            { SaslMechanism::GSSAPI, SaslMechanism::PLAIN },
+            false,
+            RpcEncryption::OPTIONAL,
+          },
+          EndpointConfig {
+            PkiConfig::NONE,
+            { SaslMechanism::GSSAPI, SaslMechanism::PLAIN },
+            false,
+            RpcEncryption::DISABLED,
+          },
+          false,
+          false,
+          Status::OK(),
+          Status::OK(),
+          AuthenticationType::SASL,
+          SaslMechanism::GSSAPI,
+          false,
+        },
+
+        // client: GSSAPI, PLAIN
+        // server: GSSAPI
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::NONE,
+            { SaslMechanism::GSSAPI, SaslMechanism::PLAIN },
+            false,
+            RpcEncryption::OPTIONAL,
+          },
+          EndpointConfig {
+            PkiConfig::NONE,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::DISABLED,
+          },
+          false,
+          false,
+          Status::OK(),
+          Status::OK(),
+          AuthenticationType::SASL,
+          SaslMechanism::GSSAPI,
+          false,
+        },
+
+        // client: PLAIN
+        // server: GSSAPI
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::NONE,
+            { SaslMechanism::PLAIN },
+            false,
+            RpcEncryption::OPTIONAL,
+          },
+          EndpointConfig {
+            PkiConfig::NONE,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::DISABLED,
+          },
+          false,
+          false,
+          Status::NotAuthorized(".*client does not have Kerberos credentials available"),
+          Status::NetworkError(""),
+          AuthenticationType::INVALID,
+          SaslMechanism::INVALID,
+          false,
+        },
+
+        // client: GSSAPI,
+        // server: GSSAPI, self-signed cert
+        // loopback encryption
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::NONE,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::OPTIONAL,
+          },
+          EndpointConfig {
+            PkiConfig::SELF_SIGNED,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::OPTIONAL,
+          },
+          false,
+          true,
+          Status::OK(),
+          Status::OK(),
+          AuthenticationType::SASL,
+          SaslMechanism::GSSAPI,
+          true,
+        },
+
+        // client: GSSAPI, signed-cert
+        // server: GSSAPI, self-signed cert
+        // This tests that the server will not advertise CERTIFICATE authentication,
+        // since it doesn't have a trusted cert.
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::SIGNED,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::OPTIONAL,
+          },
+          EndpointConfig {
+            PkiConfig::SELF_SIGNED,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::OPTIONAL,
+          },
+          false,
+          false,
+          Status::OK(),
+          Status::OK(),
+          AuthenticationType::SASL,
+          SaslMechanism::GSSAPI,
+          true,
+        },
+
+        // client: PLAIN,
+        // server: PLAIN, self-signed cert
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::NONE,
+            { SaslMechanism::PLAIN },
+            false,
+            RpcEncryption::OPTIONAL,
+          },
+          EndpointConfig {
+            PkiConfig::SELF_SIGNED,
+            { SaslMechanism::PLAIN },
+            false,
+            RpcEncryption::OPTIONAL,
+          },
+          false,
+          false,
+          Status::OK(),
+          Status::OK(),
+          AuthenticationType::SASL,
+          SaslMechanism::PLAIN,
+          true,
+        },
+
+        // client: signed-cert
+        // server: signed-cert
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::SIGNED,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::OPTIONAL,
+          },
+          EndpointConfig {
+            PkiConfig::SIGNED,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::OPTIONAL,
+          },
+          false,
+          false,
+          Status::OK(),
+          Status::OK(),
+          AuthenticationType::CERTIFICATE,
+          SaslMechanism::INVALID,
+          true,
+        },
+
+        // client: token, trusted cert
+        // server: token, signed-cert, GSSAPI
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::TRUSTED,
+            { },
+            true,
+            RpcEncryption::OPTIONAL,
+          },
+          EndpointConfig {
+            PkiConfig::SIGNED,
+            { SaslMechanism::PLAIN },
+            true,
+            RpcEncryption::OPTIONAL,
+          },
+          false,
+          false,
+          Status::OK(),
+          Status::OK(),
+          AuthenticationType::TOKEN,
+          SaslMechanism::INVALID,
+          true,
+        },
+
+        // client: PLAIN, token
+        // server: PLAIN, token, signed cert
+        // Test that the client won't negotiate token authn if it doesn't have a
+        // trusted cert. We aren't expecting this to happen in practice (the
+        // token and trusted CA cert should come as a package).
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::NONE,
+            { SaslMechanism::PLAIN },
+            true,
+            RpcEncryption::OPTIONAL,
+          },
+          EndpointConfig {
+            PkiConfig::SIGNED,
+            { SaslMechanism::PLAIN },
+            true,
+            RpcEncryption::OPTIONAL,
+          },
+          false,
+          false,
+          Status::OK(),
+          Status::OK(),
+          AuthenticationType::SASL,
+          SaslMechanism::PLAIN,
+          true,
+        },
+
+        // client: PLAIN, GSSAPI, signed-cert, token
+        // server: PLAIN, GSSAPI, signed-cert, token
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::SIGNED,
+            { SaslMechanism::PLAIN, SaslMechanism::GSSAPI },
+            true,
+            RpcEncryption::OPTIONAL,
+          },
+          EndpointConfig {
+            PkiConfig::SIGNED,
+            { SaslMechanism::PLAIN, SaslMechanism::GSSAPI },
+            true,
+            RpcEncryption::OPTIONAL,
+          },
+          false,
+          false,
+          Status::OK(),
+          Status::OK(),
+          AuthenticationType::CERTIFICATE,
+          SaslMechanism::INVALID,
+          true,
+        },
+
+        // client: PLAIN, TLS disabled
+        // server: PLAIN, TLS required
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::NONE,
+            { SaslMechanism::PLAIN },
+            false,
+            RpcEncryption::DISABLED,
+          },
+          EndpointConfig {
+            PkiConfig::SIGNED,
+            { SaslMechanism::PLAIN },
+            false,
+            RpcEncryption::REQUIRED,
+          },
+          false,
+          false,
+          Status::NotAuthorized(".*client does not support required TLS encryption"),
+          Status::NotAuthorized(".*client does not support required TLS encryption"),
+          AuthenticationType::SASL,
+          SaslMechanism::PLAIN,
+          true,
+        },
+
+        // client: PLAIN, TLS required
+        // server: PLAIN, TLS disabled
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::NONE,
+            { SaslMechanism::PLAIN },
+            false,
+            RpcEncryption::REQUIRED,
+          },
+          EndpointConfig {
+            PkiConfig::SIGNED,
+            { SaslMechanism::PLAIN },
+            false,
+            RpcEncryption::DISABLED,
+          },
+          false,
+          false,
+          Status::NotAuthorized(".*server does not support required TLS encryption"),
+          Status::NetworkError(""),
+          AuthenticationType::SASL,
+          SaslMechanism::PLAIN,
+          true,
+        },
+
+        // client: GSSAPI, TLS required, externally-signed cert
+        // server: GSSAPI, TLS required, externally-signed cert
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::EXTERNALLY_SIGNED,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::REQUIRED,
+          },
+          EndpointConfig {
+            PkiConfig::EXTERNALLY_SIGNED,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::REQUIRED,
+          },
+          false,
+          false,
+          Status::OK(),
+          Status::OK(),
+          AuthenticationType::SASL,
+          SaslMechanism::GSSAPI,
+          true,
+        },
+
+        // client: GSSAPI, TLS optional, externally-signed cert
+        // server: GSSAPI, TLS required, signed cert
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::EXTERNALLY_SIGNED,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::OPTIONAL,
+          },
+          EndpointConfig {
+            PkiConfig::SIGNED,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::REQUIRED,
+          },
+          false,
+          false,
+          Status::OK(),
+          Status::OK(),
+          AuthenticationType::SASL,
+          SaslMechanism::GSSAPI,
+          true,
+        },
+
+        // client: GSSAPI, TLS required
+        // server: GSSAPI, TLS required, externally-signed cert
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::NONE,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::REQUIRED,
+          },
+          EndpointConfig {
+            PkiConfig::EXTERNALLY_SIGNED,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::REQUIRED,
+          },
+          false,
+          false,
+          Status::OK(),
+          Status::OK(),
+          AuthenticationType::SASL,
+          SaslMechanism::GSSAPI,
+          true,
+        },
+
+        // client: GSSAPI, PLAIN, TLS required, externally-signed cert
+        // server: PLAIN, TLS required, externally-signed cert
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::EXTERNALLY_SIGNED,
+            { SaslMechanism::GSSAPI, SaslMechanism::PLAIN },
+            false,
+            RpcEncryption::REQUIRED,
+          },
+          EndpointConfig {
+            PkiConfig::EXTERNALLY_SIGNED,
+            { SaslMechanism::PLAIN },
+            false,
+            RpcEncryption::REQUIRED,
+          },
+          false,
+          false,
+          Status::OK(),
+          Status::OK(),
+          AuthenticationType::SASL,
+          SaslMechanism::PLAIN,
+          true,
+        },
+
+        // client: GSSAPI, TLS disabled, signed cert
+        // server: GSSAPI, TLS required, externally-signed cert
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::SIGNED,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::DISABLED,
+          },
+          EndpointConfig {
+            PkiConfig::EXTERNALLY_SIGNED,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::REQUIRED,
+          },
+          false,
+          false,
+          Status::NotAuthorized(".*client does not support required TLS encryption"),
+          Status::NotAuthorized(".*client does not support required TLS encryption"),
+          AuthenticationType::SASL,
+          SaslMechanism::GSSAPI,
+          true,
+        },
+
+        // client: GSSAPI, TLS required, signed cert
+        // server: GSSAPI, TLS required, externally-signed cert
+        NegotiationDescriptor {
+          EndpointConfig {
+            PkiConfig::SIGNED,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::REQUIRED,
+          },
+          EndpointConfig {
+            PkiConfig::EXTERNALLY_SIGNED,
+            { SaslMechanism::GSSAPI },
+            false,
+            RpcEncryption::REQUIRED,
+          },
+          false,
+          false,
+          Status::OK(),
+          Status::OK(),
+          AuthenticationType::SASL,
+          SaslMechanism::GSSAPI,
+          true,
+        },
+
+        // client: PLAIN
+        // server: PLAIN
+        // connection from public routable IP
+        NegotiationDescriptor {
+            EndpointConfig {
+                PkiConfig::NONE,
+                { SaslMechanism::PLAIN },
+                false,
+                RpcEncryption::OPTIONAL
+            },
+            EndpointConfig {
+                PkiConfig::NONE,
+                { SaslMechanism::PLAIN },
+                false,
+                RpcEncryption::OPTIONAL
+            },
+            true,
+            false,
+            Status::NotAuthorized(".*unencrypted connections from publicly routable IPs"),
+            Status::NotAuthorized(".*unencrypted connections from publicly routable IPs"),
+            AuthenticationType::SASL,
+            SaslMechanism::PLAIN,
+            false,
+        },
+
+        // client: GSSAPI, TLS required, externally-signed cert
+        // server: GSSAPI, TLS required, externally-signed cert
+        // connection from public routable IP
+        NegotiationDescriptor {
+            EndpointConfig {
+                PkiConfig::EXTERNALLY_SIGNED,
+                { SaslMechanism::GSSAPI },
+                false,
+                RpcEncryption::REQUIRED,
+            },
+            EndpointConfig {
+                PkiConfig::EXTERNALLY_SIGNED,
+                { SaslMechanism::GSSAPI },
+                false,
+                RpcEncryption::REQUIRED,
+            },
+            true,
+            // true as no longer a loopback connection.
+            true,
+            Status::OK(),
+            Status::OK(),
+            AuthenticationType::SASL,
+            SaslMechanism::GSSAPI,
+            true,
+        }
+));
+
+// A "Callable" that takes a socket for use with starting a thread.
+// Can be used for ServerNegotiation or ClientNegotiation threads.
+typedef std::function<void(unique_ptr<Socket>)> SocketCallable;
+
+// Call Accept() on the socket, then pass the connection to the server runner
+static void RunAcceptingDelegator(Socket* acceptor,
+                                  const SocketCallable& server_runner) {
+  unique_ptr<Socket> conn(new Socket());
+  Sockaddr remote;
+  CHECK_OK(acceptor->Accept(conn.get(), &remote, 0));
+  server_runner(std::move(conn));
+}
+
+// Set up a socket and run a negotiation sequence.
+static void RunNegotiationTest(const SocketCallable& server_runner,
+                               const SocketCallable& client_runner) {
+  Socket server_sock;
+  CHECK_OK(server_sock.Init(0));
+  ASSERT_OK(server_sock.BindAndListen(Sockaddr(), 1));
+  Sockaddr server_bind_addr;
+  ASSERT_OK(server_sock.GetSocketAddress(&server_bind_addr));
+  thread server(RunAcceptingDelegator, &server_sock, server_runner);
+
+  unique_ptr<Socket> client_sock(new Socket());
+  CHECK_OK(client_sock->Init(0));
+  ASSERT_OK(client_sock->Connect(server_bind_addr));
+  thread client(client_runner, std::move(client_sock));
+
+  LOG(INFO) << "Waiting for test threads to terminate...";
+  client.join();
+  LOG(INFO) << "Client thread terminated.";
+
+  server.join();
+  LOG(INFO) << "Server thread terminated.";
+}
+
+////////////////////////////////////////////////////////////////////////////////
+
+#ifndef __APPLE__
+template<class T>
+using CheckerFunction = std::function<void(const Status&, T&)>;
+
+// Run GSSAPI negotiation from the server side. Runs
+// 'post_check' after negotiation to verify the result.
+static void RunGSSAPINegotiationServer(unique_ptr<Socket> socket,
+                                       const CheckerFunction<ServerNegotiation>& post_check) {
+  TlsContext tls_context;
+  CHECK_OK(tls_context.Init());
+  TokenVerifier token_verifier;
+  ServerNegotiation server_negotiation(std::move(socket), &tls_context,
+                                       &token_verifier, RpcEncryption::OPTIONAL, "kudu");
+  server_negotiation.set_server_fqdn("127.0.0.1");
+  CHECK_OK(server_negotiation.EnableGSSAPI());
+  post_check(server_negotiation.Negotiate(), server_negotiation);
+}
+
+// Run GSSAPI negotiation from the client side. Runs
+// 'post_check' after negotiation to verify the result.
+static void RunGSSAPINegotiationClient(unique_ptr<Socket> conn,
+                                       const CheckerFunction<ClientNegotiation>& post_check) {
+  TlsContext tls_context;
+  CHECK_OK(tls_context.Init());
+  ClientNegotiation client_negotiation(std::move(conn), &tls_context,
+                                       boost::none, RpcEncryption::OPTIONAL, "kudu");
+  client_negotiation.set_server_fqdn("127.0.0.1");
+  CHECK_OK(client_negotiation.EnableGSSAPI());
+  post_check(client_negotiation.Negotiate(), client_negotiation);
+}
+
+// Test invalid SASL negotiations using the GSSAPI (kerberos) mechanism over a socket.
+// This test is ignored on macOS because the system Kerberos implementation
+// (Heimdal) caches the non-existence of client credentials, which causes futher
+// tests to fail.
+TEST_F(TestNegotiation, TestGSSAPIInvalidNegotiation) {
+  MiniKdc kdc;
+  ASSERT_OK(kdc.Start());
+
+  // Try to negotiate with no krb5 credentials on either side. It should fail on both
+  // sides.
+  RunNegotiationTest(
+      std::bind(RunGSSAPINegotiationServer, std::placeholders::_1,
+                [](const Status& s, ServerNegotiation& server) {
+                  // The client notices there are no credentials and
+                  // doesn't send any failure message to the server.
+                  // Instead, it just disconnects.
+                  //
+                  // TODO(todd): it might be preferable to have the server
+                  // fail to start if it has no valid keytab.
+                  CHECK(s.IsNetworkError());
+                }),
+      std::bind(RunGSSAPINegotiationClient, std::placeholders::_1,
+                [](const Status& s, ClientNegotiation& client) {
+                  CHECK(s.IsNotAuthorized());
+#ifndef KRB5_VERSION_LE_1_10
+                  CHECK_GT(s.ToString().find("No Kerberos credentials available"), 0);
+#endif
+                }));
+
+  // Create the server principal and keytab.
+  string kt_path;
+  ASSERT_OK(kdc.CreateServiceKeytab("kudu/127.0.0.1", &kt_path));
+  CHECK_ERR(setenv("KRB5_KTNAME", kt_path.c_str(), 1 /*replace*/));
+
+  // Try to negotiate with no krb5 credentials on the client. It should fail on both
+  // sides.
+  RunNegotiationTest(
+      std::bind(RunGSSAPINegotiationServer, std::placeholders::_1,
+                [](const Status& s, ServerNegotiation& server) {
+                  // The client notices there are no credentials and
+                  // doesn't send any failure message to the server.
+                  // Instead, it just disconnects.
+                  CHECK(s.IsNetworkError());
+                }),
+      std::bind(RunGSSAPINegotiationClient, std::placeholders::_1,
+                [](const Status& s, ClientNegotiation& client) {
+                  CHECK(s.IsNotAuthorized());
+                  ASSERT_STR_MATCHES(s.ToString(),
+                                     "Not authorized: server requires authentication, "
+                                     "but client does not have Kerberos credentials available");
+                }));
+
+  // Create and kinit as a client user.
+  ASSERT_OK(kdc.CreateUserPrincipal("testuser"));
+  ASSERT_OK(kdc.Kinit("testuser"));
+  ASSERT_OK(kdc.SetKrb5Environment());
+
+  // Change the server's keytab file so that it has inappropriate
+  // credentials.
+  // Authentication should now fail.
+  ASSERT_OK(kdc.CreateServiceKeytab("otherservice/127.0.0.1", &kt_path));
+  CHECK_ERR(setenv("KRB5_KTNAME", kt_path.c_str(), 1 /*replace*/));
+
+  RunNegotiationTest(
+      std::bind(RunGSSAPINegotiationServer, std::placeholders::_1,
+                [](const Status& s, ServerNegotiation& server) {
+                  CHECK(s.IsNotAuthorized());
+#ifndef KRB5_VERSION_LE_1_10
+                  ASSERT_STR_CONTAINS(s.ToString(),
+                                      "No key table entry found matching kudu/127.0.0.1");
+#endif
+                }),
+      std::bind(RunGSSAPINegotiationClient, std::placeholders::_1,
+                [](const Status& s, ClientNegotiation& client) {
+                  CHECK(s.IsNotAuthorized());
+#ifndef KRB5_VERSION_LE_1_10
+                  ASSERT_STR_CONTAINS(s.ToString(),
+                                      "No key table entry found matching kudu/127.0.0.1");
+#endif
+                }));
+}
+#endif
+
+#ifndef __APPLE__
+// Test that the pre-flight check for servers requiring Kerberos provides
+// nice error messages for missing or bad keytabs.
+//
+// This is ignored on macOS because the system Kerberos implementation does not
+// fail the preflight check when the keytab is inaccessible, probably because
+// the preflight check passes a 0-length token.
+TEST_F(TestNegotiation, TestPreflight) {
+  // Try pre-flight with no keytab.
+  Status s = ServerNegotiation::PreflightCheckGSSAPI("kudu");
+  ASSERT_FALSE(s.ok());
+#ifndef KRB5_VERSION_LE_1_10
+  ASSERT_STR_MATCHES(s.ToString(), "Key table file.*not found");
+#endif
+  // Try with a valid krb5 environment and keytab.
+  MiniKdc kdc;
+  ASSERT_OK(kdc.Start());
+  ASSERT_OK(kdc.SetKrb5Environment());
+  string kt_path;
+  ASSERT_OK(kdc.CreateServiceKeytab("kudu/127.0.0.1", &kt_path));
+  CHECK_ERR(setenv("KRB5_KTNAME", kt_path.c_str(), 1 /*replace*/));
+
+  ASSERT_OK(ServerNegotiation::PreflightCheckGSSAPI("kudu"));
+
+  // Try with an inaccessible keytab.
+  CHECK_ERR(chmod(kt_path.c_str(), 0000));
+  s = ServerNegotiation::PreflightCheckGSSAPI("kudu");
+  ASSERT_FALSE(s.ok());
+#ifndef KRB5_VERSION_LE_1_10
+  ASSERT_STR_MATCHES(s.ToString(), "error accessing keytab: Permission denied");
+#endif
+  CHECK_ERR(unlink(kt_path.c_str()));
+
+  // Try with a keytab that has the wrong credentials.
+  ASSERT_OK(kdc.CreateServiceKeytab("wrong-service/127.0.0.1", &kt_path));
+  CHECK_ERR(setenv("KRB5_KTNAME", kt_path.c_str(), 1 /*replace*/));
+  s = ServerNegotiation::PreflightCheckGSSAPI("kudu");
+  ASSERT_FALSE(s.ok());
+#ifndef KRB5_VERSION_LE_1_10
+  ASSERT_STR_MATCHES(s.ToString(), "No key table entry found matching kudu/.*");
+#endif
+}
+#endif
+
+////////////////////////////////////////////////////////////////////////////////
+
+static void RunTimeoutExpectingServer(unique_ptr<Socket> socket) {
+  TlsContext tls_context;
+  CHECK_OK(tls_context.Init());
+  TokenVerifier token_verifier;
+  ServerNegotiation server_negotiation(std::move(socket), &tls_context,
+                                       &token_verifier, RpcEncryption::OPTIONAL, "kudu");
+  CHECK_OK(server_negotiation.EnablePlain());
+  Status s = server_negotiation.Negotiate();
+  ASSERT_TRUE(s.IsNetworkError()) << "Expected client to time out and close the connection. Got: "
+                                  << s.ToString();
+}
+
+static void RunTimeoutNegotiationClient(unique_ptr<Socket> sock) {
+  TlsContext tls_context;
+  CHECK_OK(tls_context.Init());
+  ClientNegotiation client_negotiation(std::move(sock), &tls_context,
+                                       boost::none, RpcEncryption::OPTIONAL, "kudu");
+  CHECK_OK(client_negotiation.EnablePlain("test", "test"));
+  MonoTime deadline = MonoTime::Now() - MonoDelta::FromMilliseconds(100L);
+  client_negotiation.set_deadline(deadline);
+  Status s = client_negotiation.Negotiate();
+  ASSERT_TRUE(s.IsTimedOut()) << "Expected timeout! Got: " << s.ToString();
+  CHECK_OK(client_negotiation.socket()->Shutdown(true, true));
+}
+
+// Ensure that the client times out.
+TEST_F(TestNegotiation, TestClientTimeout) {
+  RunNegotiationTest(RunTimeoutExpectingServer, RunTimeoutNegotiationClient);
+}
+
+////////////////////////////////////////////////////////////////////////////////
+
+static void RunTimeoutNegotiationServer(unique_ptr<Socket> socket) {
+  TlsContext tls_context;
+  CHECK_OK(tls_context.Init());
+  TokenVerifier token_verifier;
+  ServerNegotiation server_negotiation(std::move(socket), &tls_context,
+                                       &token_verifier, RpcEncryption::OPTIONAL, "kudu");
+  CHECK_OK(server_negotiation.EnablePlain());
+  MonoTime deadline = MonoTime::Now() - MonoDelta::FromMilliseconds(100L);
+  server_negotiation.set_deadline(deadline);
+  Status s = server_negotiation.Negotiate();
+  ASSERT_TRUE(s.IsTimedOut()) << "Expected timeout! Got: " << s.ToString();
+  CHECK_OK(server_negotiation.socket()->Close());
+}
+
+static void RunTimeoutExpectingClient(unique_ptr<Socket> socket) {
+  TlsContext tls_context;
+  CHECK_OK(tls_context.Init());
+  ClientNegotiation client_negotiation(std::move(socket), &tls_context,
+                                       boost::none, RpcEncryption::OPTIONAL, "kudu");
+  CHECK_OK(client_negotiation.EnablePlain("test", "test"));
+  Status s = client_negotiation.Negotiate();
+  ASSERT_TRUE(s.IsNetworkError()) << "Expected server to time out and close the connection. Got: "
+      << s.ToString();
+}
+
+// Ensure that the server times out.
+TEST_F(TestNegotiation, TestServerTimeout) {
+  RunNegotiationTest(RunTimeoutNegotiationServer, RunTimeoutExpectingClient);
+}
+
+////////////////////////////////////////////////////////////////////////////////
+
+// This suite of tests ensure that applications that embed the Kudu client are
+// able to externally handle the initialization of SASL. See KUDU-1749 and
+// IMPALA-4497 for context.
+//
+// The tests are a bit tricky because the initialization of SASL is static state
+// that we can't easily clear/reset between test cases. So, each test invokes
+// itself as a subprocess with the appropriate --gtest_filter line as well as a
+// special flag to indicate that it is the test child running.
+class TestDisableInit : public KuduTest {
+ protected:
+  // Run the lambda 'f' in a newly-started process, capturing its stderr
+  // into 'stderr'.
+  template<class TestFunc>
+  void DoTest(const TestFunc& f, string* stderr = nullptr) {
+    if (FLAGS_is_test_child) {
+      f();
+      return;
+    }
+
+    // Invoke the currently-running test case in a new subprocess.
+    string filter_flag = strings::Substitute("--gtest_filter=$0.$1",
+                                             CURRENT_TEST_CASE_NAME(), CURRENT_TEST_NAME());
+    string executable_path;
+    CHECK_OK(env_->GetExecutablePath(&executable_path));
+    string stdout;
+    Status s = Subprocess::Call({ executable_path, "test", filter_flag, "--is_test_child" },
+                                "" /* stdin */,
+                                &stdout,
+                                stderr);
+    ASSERT_TRUE(s.ok()) << "Test failed: " << stdout;
+  }
+};
+
+// Test disabling SASL but not actually properly initializing it before usage.
+TEST_F(TestDisableInit, TestDisableSasl_NotInitialized) {
+  DoTest([]() {
+      CHECK_OK(DisableSaslInitialization());
+      Status s = SaslInit();
+      ASSERT_STR_CONTAINS(s.ToString(), "was disabled, but SASL was not externally initialized");
+    });
+}
+
+// Test disabling SASL with proper initialization by some other app.
+TEST_F(TestDisableInit, TestDisableSasl_Good) {
+  DoTest([]() {
+      rpc::internal::SaslSetMutex();
+      sasl_client_init(NULL);
+      CHECK_OK(DisableSaslInitialization());
+      ASSERT_OK(SaslInit());
+    });
+}
+
+// Test a client which inits SASL itself but doesn't remember to disable Kudu's
+// SASL initialization.
+TEST_F(TestDisableInit, TestMultipleSaslInit) {
+  string stderr;
+  DoTest([]() {
+      rpc::internal::SaslSetMutex();
+      sasl_client_init(NULL);
+      ASSERT_OK(SaslInit());
+    }, &stderr);
+  // If we are the parent, we should see the warning from the child that it automatically
+  // skipped initialization because it detected that it was already initialized.
+  if (!FLAGS_is_test_child) {
+    ASSERT_STR_CONTAINS(stderr, "Skipping initialization");
+  }
+}
+
+// We are not able to detect mutexes not being set with the macOS version of libsasl.
+#ifndef __APPLE__
+// Test disabling SASL but not remembering to initialize the SASL mutex support. This
+// should succeed but generate a warning.
+TEST_F(TestDisableInit, TestDisableSasl_NoMutexImpl) {
+  string stderr;
+  DoTest([]() {
+      sasl_client_init(NULL);
+      CHECK_OK(DisableSaslInitialization());
+      ASSERT_OK(SaslInit());
+    }, &stderr);
+  // If we are the parent, we should see the warning from the child.
+  if (!FLAGS_is_test_child) {
+    ASSERT_STR_CONTAINS(stderr, "not provided with a mutex implementation");
+  }
+}
+
+// Test a client which inits SASL itself but doesn't remember to disable Kudu's
+// SASL initialization.
+TEST_F(TestDisableInit, TestMultipleSaslInit_NoMutexImpl) {
+  string stderr;
+  DoTest([]() {
+      sasl_client_init(NULL);
+      ASSERT_OK(SaslInit());
+    }, &stderr);
+  // If we are the parent, we should see the warning from the child that it automatically
+  // skipped initialization because it detected that it was already initialized.
+  if (!FLAGS_is_test_child) {
+    ASSERT_STR_CONTAINS(stderr, "Skipping initialization");
+    ASSERT_STR_CONTAINS(stderr, "not provided with a mutex implementation");
+  }
+}
+#endif
+
+} // namespace rpc
+} // namespace kudu
diff --git a/be/src/kudu/rpc/negotiation.cc b/be/src/kudu/rpc/negotiation.cc
new file mode 100644
index 0000000..31e0b33
--- /dev/null
+++ b/be/src/kudu/rpc/negotiation.cc
@@ -0,0 +1,327 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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.
+
+#include "kudu/rpc/negotiation.h"
+
+#include <poll.h>
+#include <sys/socket.h>
+
+#include <cerrno>
+#include <ctime>
+#include <memory>
+#include <ostream>
+#include <string>
+#include <utility>
+
+#include <boost/optional/optional.hpp>
+#include <gflags/gflags.h>
+#include <glog/logging.h>
+
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/rpc/client_negotiation.h"
+#include "kudu/rpc/connection.h"
+#include "kudu/rpc/connection_id.h"
+#include "kudu/rpc/messenger.h"
+#include "kudu/rpc/reactor.h"
+#include "kudu/rpc/rpc_controller.h"
+#include "kudu/rpc/rpc_header.pb.h"
+#include "kudu/rpc/server_negotiation.h"
+#include "kudu/rpc/user_credentials.h"
+#include "kudu/security/tls_context.h"
+#include "kudu/security/token.pb.h"
+#include "kudu/util/errno.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/logging.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/net/socket.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+#include "kudu/util/trace.h"
+
+DEFINE_bool(rpc_trace_negotiation, false,
+            "If enabled, dump traces of all RPC negotiations to the log");
+TAG_FLAG(rpc_trace_negotiation, runtime);
+TAG_FLAG(rpc_trace_negotiation, advanced);
+TAG_FLAG(rpc_trace_negotiation, experimental);
+
+DEFINE_int32(rpc_negotiation_inject_delay_ms, 0,
+             "If enabled, injects the given number of milliseconds delay into "
+             "the RPC negotiation process on the server side.");
+TAG_FLAG(rpc_negotiation_inject_delay_ms, unsafe);
+
+DEFINE_bool(rpc_encrypt_loopback_connections, false,
+            "Whether to encrypt data transfer on RPC connections that stay within "
+            "a single host. Encryption here is likely to offer no additional "
+            "security benefit since only a local 'root' user could intercept the "
+            "traffic, and wire encryption does not suitably protect against such "
+            "an attacker.");
+TAG_FLAG(rpc_encrypt_loopback_connections, advanced);
+
+using std::string;
+using std::unique_ptr;
+using strings::Substitute;
+
+namespace kudu {
+namespace rpc {
+
+const char* AuthenticationTypeToString(AuthenticationType t) {
+  switch (t) {
+    case AuthenticationType::INVALID: return "INVALID"; break;
+    case AuthenticationType::SASL: return "SASL"; break;
+    case AuthenticationType::TOKEN: return "TOKEN"; break;
+    case AuthenticationType::CERTIFICATE: return "CERTIFICATE"; break;
+  }
+  return "<cannot reach here>";
+}
+
+std::ostream& operator<<(std::ostream& o, AuthenticationType authentication_type) {
+  return o << AuthenticationTypeToString(authentication_type);
+}
+
+// Wait for the client connection to be established and become ready for writing.
+static Status WaitForClientConnect(Socket* socket, const MonoTime& deadline) {
+  TRACE("Waiting for socket to connect");
+  int fd = socket->GetFd();
+  struct pollfd poll_fd;
+  poll_fd.fd = fd;
+  poll_fd.events = POLLOUT;
+  poll_fd.revents = 0;
+
+  MonoTime now;
+  MonoDelta remaining;
+  while (true) {
+    now = MonoTime::Now();
+    remaining = deadline - now;
+    DVLOG(4) << "Client waiting to connect for negotiation, time remaining until timeout deadline: "
+             << remaining.ToString();
+    if (PREDICT_FALSE(remaining.ToNanoseconds() <= 0)) {
+      return Status::TimedOut("Timeout exceeded waiting to connect");
+    }
+#if defined(__linux__)
+    struct timespec ts;
+    remaining.ToTimeSpec(&ts);
+    int ready = ppoll(&poll_fd, 1, &ts, NULL);
+#else
+    int ready = poll(&poll_fd, 1, remaining.ToMilliseconds());
+#endif
+    if (ready == -1) {
+      int err = errno;
+      if (err == EINTR) {
+        // We were interrupted by a signal, let's go again.
+        continue;
+      } else {
+        return Status::NetworkError("Error from ppoll() while waiting to connect",
+            ErrnoToString(err), err);
+      }
+    } else if (ready == 0) {
+      // Timeout exceeded. Loop back to the top to our impending doom.
+      continue;
+    } else {
+      // Success.
+      break;
+    }
+  }
+
+  // Connect finished, but this doesn't mean that we connected successfully.
+  // Check the socket for an error.
+  int so_error = 0;
+  socklen_t socklen = sizeof(so_error);
+  int rc = getsockopt(fd, SOL_SOCKET, SO_ERROR, &so_error, &socklen);
+  if (rc != 0) {
+    return Status::NetworkError("Unable to check connected socket for errors",
+                                ErrnoToString(errno),
+                                errno);
+  }
+  if (so_error != 0) {
+    return Status::NetworkError("connect", ErrnoToString(so_error), so_error);
+  }
+
+  return Status::OK();
+}
+
+// Disable / reset socket timeouts.
+static Status DisableSocketTimeouts(Socket* socket) {
+  RETURN_NOT_OK(socket->SetSendTimeout(MonoDelta::FromNanoseconds(0L)));
+  RETURN_NOT_OK(socket->SetRecvTimeout(MonoDelta::FromNanoseconds(0L)));
+  return Status::OK();
+}
+
+// Perform client negotiation. We don't LOG() anything, we leave that to our caller.
+static Status DoClientNegotiation(Connection* conn,
+                                  RpcAuthentication authentication,
+                                  RpcEncryption encryption,
+                                  MonoTime deadline,
+                                  unique_ptr<ErrorStatusPB>* rpc_error) {
+  const auto* messenger = conn->reactor_thread()->reactor()->messenger();
+  // Prefer secondary credentials (such as authn token) if permitted by policy.
+  const auto authn_token = (conn->credentials_policy() == CredentialsPolicy::PRIMARY_CREDENTIALS)
+      ? boost::none : messenger->authn_token();
+  ClientNegotiation client_negotiation(conn->release_socket(),
+                                       &messenger->tls_context(),
+                                       authn_token,
+                                       encryption,
+                                       messenger->sasl_proto_name());
+
+  client_negotiation.set_server_fqdn(conn->outbound_connection_id().hostname());
+
+  if (authentication != RpcAuthentication::DISABLED) {
+    Status s = client_negotiation.EnableGSSAPI();
+    if (!s.ok()) {
+      // If we can't enable GSSAPI, it's likely the client is just missing the
+      // appropriate SASL plugin. We don't want to require it to be installed
+      // if the user doesn't care about connecting to servers using Kerberos
+      // authentication. So, we'll just VLOG this here. If we try to connect
+      // to a server which requires Kerberos, we'll get a negotiation error
+      // at that point.
+      if (VLOG_IS_ON(1)) {
+        KLOG_FIRST_N(INFO, 1) << "Couldn't enable GSSAPI (Kerberos) SASL plugin: "
+                              << s.message().ToString()
+                              << ". This process will be unable to connect to "
+                              << "servers requiring Kerberos authentication.";
+      }
+
+      if (authentication == RpcAuthentication::REQUIRED &&
+          !authn_token &&
+          !messenger->tls_context().has_signed_cert()) {
+        return Status::InvalidArgument(
+            "Kerberos, token, or PKI certificate credentials must be provided in order to "
+            "require authentication for a client");
+      }
+    }
+  }
+
+  if (authentication != RpcAuthentication::REQUIRED) {
+    const auto& creds = conn->outbound_connection_id().user_credentials();
+    RETURN_NOT_OK(client_negotiation.EnablePlain(creds.real_user(), ""));
+  }
+
+  client_negotiation.set_deadline(deadline);
+
+  RETURN_NOT_OK(WaitForClientConnect(client_negotiation.socket(), deadline));
+  RETURN_NOT_OK(client_negotiation.socket()->SetNonBlocking(false));
+  RETURN_NOT_OK(client_negotiation.Negotiate(rpc_error));
+  RETURN_NOT_OK(DisableSocketTimeouts(client_negotiation.socket()));
+
+  // Transfer the negotiated socket and state back to the connection.
+  conn->adopt_socket(client_negotiation.release_socket());
+  conn->set_remote_features(client_negotiation.take_server_features());
+  conn->set_confidential(client_negotiation.tls_negotiated() ||
+      (conn->socket()->IsLoopbackConnection() && !FLAGS_rpc_encrypt_loopback_connections));
+
+  // Sanity check: if no authn token was supplied as user credentials,
+  // the negotiated authentication type cannot be AuthenticationType::TOKEN.
+  DCHECK(!(authn_token == boost::none &&
+           client_negotiation.negotiated_authn() == AuthenticationType::TOKEN));
+
+  return Status::OK();
+}
+
+// Perform server negotiation. We don't LOG() anything, we leave that to our caller.
+static Status DoServerNegotiation(Connection* conn,
+                                  RpcAuthentication authentication,
+                                  RpcEncryption encryption,
+                                  const MonoTime& deadline) {
+  const auto* messenger = conn->reactor_thread()->reactor()->messenger();
+  if (authentication == RpcAuthentication::REQUIRED &&
+      messenger->keytab_file().empty() &&
+      !messenger->tls_context().is_external_cert()) {
+    return Status::InvalidArgument("RPC authentication (--rpc_authentication) may not be "
+                                   "required unless Kerberos (--keytab_file) or external PKI "
+                                   "(--rpc_certificate_file et al) are configured");
+  }
+
+  if (FLAGS_rpc_negotiation_inject_delay_ms > 0) {
+    LOG(WARNING) << "Injecting " << FLAGS_rpc_negotiation_inject_delay_ms
+                 << "ms delay in negotiation";
+    SleepFor(MonoDelta::FromMilliseconds(FLAGS_rpc_negotiation_inject_delay_ms));
+  }
+
+  // Create a new ServerNegotiation to handle the synchronous negotiation.
+  ServerNegotiation server_negotiation(conn->release_socket(),
+                                       &messenger->tls_context(),
+                                       &messenger->token_verifier(),
+                                       encryption,
+                                       messenger->sasl_proto_name());
+
+  if (authentication != RpcAuthentication::DISABLED && !messenger->keytab_file().empty()) {
+    RETURN_NOT_OK(server_negotiation.EnableGSSAPI());
+  }
+  if (authentication != RpcAuthentication::REQUIRED) {
+    RETURN_NOT_OK(server_negotiation.EnablePlain());
+  }
+
+  server_negotiation.set_deadline(deadline);
+
+  RETURN_NOT_OK(server_negotiation.socket()->SetNonBlocking(false));
+
+  RETURN_NOT_OK(server_negotiation.Negotiate());
+  RETURN_NOT_OK(DisableSocketTimeouts(server_negotiation.socket()));
+
+  // Transfer the negotiated socket and state back to the connection.
+  conn->adopt_socket(server_negotiation.release_socket());
+  conn->set_remote_features(server_negotiation.take_client_features());
+  conn->set_remote_user(server_negotiation.take_authenticated_user());
+  conn->set_confidential(server_negotiation.tls_negotiated() ||
+      (conn->socket()->IsLoopbackConnection() && !FLAGS_rpc_encrypt_loopback_connections));
+
+  return Status::OK();
+}
+
+void Negotiation::RunNegotiation(const scoped_refptr<Connection>& conn,
+                                 RpcAuthentication authentication,
+                                 RpcEncryption encryption,
+                                 MonoTime deadline) {
+  Status s;
+  unique_ptr<ErrorStatusPB> rpc_error;
+  if (conn->direction() == Connection::SERVER) {
+    s = DoServerNegotiation(conn.get(), authentication, encryption, deadline);
+  } else {
+    s = DoClientNegotiation(conn.get(), authentication, encryption, deadline,
+                            &rpc_error);
+  }
+
+  if (PREDICT_FALSE(!s.ok())) {
+    string msg = Substitute("$0 connection negotiation failed: $1",
+                            conn->direction() == Connection::SERVER ? "Server" : "Client",
+                            conn->ToString());
+    s = s.CloneAndPrepend(msg);
+  }
+  TRACE("Negotiation complete: $0", s.ToString());
+
+  bool is_bad = !s.ok() && !(
+      (s.IsNetworkError() && s.posix_code() == ECONNREFUSED) ||
+      s.IsNotAuthorized());
+
+  if (is_bad || FLAGS_rpc_trace_negotiation) {
+    string msg = Trace::CurrentTrace()->DumpToString();
+    if (is_bad) {
+      LOG(WARNING) << "Failed RPC negotiation. Trace:\n" << msg;
+    } else {
+      LOG(INFO) << "RPC negotiation tracing enabled. Trace:\n" << msg;
+    }
+  }
+
+  if (conn->direction() == Connection::SERVER && s.IsNotAuthorized()) {
+    LOG(WARNING) << "Unauthorized connection attempt: " << s.message().ToString();
+  }
+  conn->CompleteNegotiation(std::move(s), std::move(rpc_error));
+}
+
+
+} // namespace rpc
+} // namespace kudu
diff --git a/be/src/kudu/rpc/negotiation.h b/be/src/kudu/rpc/negotiation.h
new file mode 100644
index 0000000..b25ed0e
--- /dev/null
+++ b/be/src/kudu/rpc/negotiation.h
@@ -0,0 +1,58 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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.
+#ifndef KUDU_RPC_NEGOTIATION_H
+#define KUDU_RPC_NEGOTIATION_H
+
+#include <iosfwd>
+
+#include "kudu/gutil/ref_counted.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/security/security_flags.h"
+
+namespace kudu {
+
+class MonoTime;
+
+namespace rpc {
+
+class Connection;
+
+enum class AuthenticationType {
+  INVALID,
+  SASL,
+  TOKEN,
+  CERTIFICATE,
+};
+const char* AuthenticationTypeToString(AuthenticationType t);
+
+std::ostream& operator<<(std::ostream& o, AuthenticationType authentication_type);
+
+class Negotiation {
+ public:
+
+  // Perform negotiation for a connection (either server or client)
+  static void RunNegotiation(const scoped_refptr<Connection>& conn,
+                             security::RpcAuthentication authentication,
+                             security::RpcEncryption encryption,
+                             MonoTime deadline);
+ private:
+  DISALLOW_IMPLICIT_CONSTRUCTORS(Negotiation);
+};
+
+} // namespace rpc
+} // namespace kudu
+#endif // KUDU_RPC_NEGOTIATION_H
diff --git a/be/src/kudu/rpc/outbound_call.cc b/be/src/kudu/rpc/outbound_call.cc
new file mode 100644
index 0000000..37d02ac
--- /dev/null
+++ b/be/src/kudu/rpc/outbound_call.cc
@@ -0,0 +1,531 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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.
+
+#include <cstdint>
+#include <memory>
+#include <mutex>
+#include <string>
+#include <type_traits>
+#include <unordered_set>
+#include <utility>
+#include <vector>
+
+#include <boost/function.hpp>
+#include <gflags/gflags.h>
+#include <google/protobuf/message.h>
+
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/stringprintf.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/sysinfo.h"
+#include "kudu/gutil/walltime.h"
+#include "kudu/rpc/constants.h"
+#include "kudu/rpc/outbound_call.h"
+#include "kudu/rpc/rpc_controller.h"
+#include "kudu/rpc/rpc_introspection.pb.h"
+#include "kudu/rpc/rpc_sidecar.h"
+#include "kudu/rpc/serialization.h"
+#include "kudu/rpc/transfer.h"
+#include "kudu/util/flag_tags.h"
+#include "kudu/util/kernel_stack_watchdog.h"
+#include "kudu/util/net/sockaddr.h"
+
+// 100M cycles should be about 50ms on a 2Ghz box. This should be high
+// enough that involuntary context switches don't trigger it, but low enough
+// that any serious blocking behavior on the reactor would.
+DEFINE_int64(rpc_callback_max_cycles, 100 * 1000 * 1000,
+             "The maximum number of cycles for which an RPC callback "
+             "should be allowed to run without emitting a warning."
+             " (Advanced debugging option)");
+TAG_FLAG(rpc_callback_max_cycles, advanced);
+TAG_FLAG(rpc_callback_max_cycles, runtime);
+
+// Flag used in debug build for injecting cancellation at different code paths.
+DEFINE_int32(rpc_inject_cancellation_state, -1,
+             "If this flag is not -1, it is the state in which a cancellation request "
+             "will be injected. Should use values in OutboundCall::State only");
+TAG_FLAG(rpc_inject_cancellation_state, unsafe);
+
+using std::string;
+using std::unique_ptr;
+using std::vector;
+
+namespace kudu {
+namespace rpc {
+
+using google::protobuf::Message;
+using strings::Substitute;
+
+static const double kMicrosPerSecond = 1000000.0;
+
+///
+/// OutboundCall
+///
+
+OutboundCall::OutboundCall(const ConnectionId& conn_id,
+                           const RemoteMethod& remote_method,
+                           google::protobuf::Message* response_storage,
+                           RpcController* controller,
+                           ResponseCallback callback)
+    : state_(READY),
+      remote_method_(remote_method),
+      conn_id_(conn_id),
+      callback_(std::move(callback)),
+      controller_(DCHECK_NOTNULL(controller)),
+      response_(DCHECK_NOTNULL(response_storage)),
+      cancellation_requested_(false) {
+  DVLOG(4) << "OutboundCall " << this << " constructed with state_: " << StateName(state_)
+           << " and RPC timeout: "
+           << (controller->timeout().Initialized() ? controller->timeout().ToString() : "none");
+  header_.set_call_id(kInvalidCallId);
+  remote_method.ToPB(header_.mutable_remote_method());
+  start_time_ = MonoTime::Now();
+
+  if (!controller_->required_server_features().empty()) {
+    required_rpc_features_.insert(RpcFeatureFlag::APPLICATION_FEATURE_FLAGS);
+  }
+
+  if (controller_->request_id_) {
+    header_.set_allocated_request_id(controller_->request_id_.release());
+  }
+}
+
+OutboundCall::~OutboundCall() {
+  DCHECK(IsFinished());
+  DVLOG(4) << "OutboundCall " << this << " destroyed with state_: " << StateName(state_);
+}
+
+size_t OutboundCall::SerializeTo(TransferPayload* slices) {
+  DCHECK_LT(0, request_buf_.size())
+      << "Must call SetRequestPayload() before SerializeTo()";
+
+  const MonoDelta &timeout = controller_->timeout();
+  if (timeout.Initialized()) {
+    header_.set_timeout_millis(timeout.ToMilliseconds());
+  }
+
+  for (uint32_t feature : controller_->required_server_features()) {
+    header_.add_required_feature_flags(feature);
+  }
+
+  DCHECK_LE(0, sidecar_byte_size_);
+  serialization::SerializeHeader(
+      header_, sidecar_byte_size_ + request_buf_.size(), &header_buf_);
+
+  size_t n_slices = 2 + sidecars_.size();
+  DCHECK_LE(n_slices, slices->size());
+  auto slice_iter = slices->begin();
+  *slice_iter++ = Slice(header_buf_);
+  *slice_iter++ = Slice(request_buf_);
+  for (auto& sidecar : sidecars_) {
+    *slice_iter++ = sidecar->AsSlice();
+  }
+  DCHECK_EQ(slice_iter - slices->begin(), n_slices);
+  return n_slices;
+}
+
+void OutboundCall::SetRequestPayload(const Message& req,
+    vector<unique_ptr<RpcSidecar>>&& sidecars) {
+  DCHECK_EQ(-1, sidecar_byte_size_);
+
+  sidecars_ = move(sidecars);
+  DCHECK_LE(sidecars_.size(), TransferLimits::kMaxSidecars);
+
+  // Compute total size of sidecar payload so that extra space can be reserved as part of
+  // the request body.
+  uint32_t message_size = req.ByteSize();
+  sidecar_byte_size_ = 0;
+  for (const unique_ptr<RpcSidecar>& car: sidecars_) {
+    header_.add_sidecar_offsets(sidecar_byte_size_ + message_size);
+    int32_t sidecar_bytes = car->AsSlice().size();
+    DCHECK_LE(sidecar_byte_size_, TransferLimits::kMaxTotalSidecarBytes - sidecar_bytes);
+    sidecar_byte_size_ += sidecar_bytes;
+  }
+
+  serialization::SerializeMessage(req, &request_buf_, sidecar_byte_size_, true);
+}
+
+Status OutboundCall::status() const {
+  std::lock_guard<simple_spinlock> l(lock_);
+  return status_;
+}
+
+const ErrorStatusPB* OutboundCall::error_pb() const {
+  std::lock_guard<simple_spinlock> l(lock_);
+  return error_pb_.get();
+}
+
+string OutboundCall::StateName(State state) {
+  switch (state) {
+    case READY:
+      return "READY";
+    case ON_OUTBOUND_QUEUE:
+      return "ON_OUTBOUND_QUEUE";
+    case SENDING:
+      return "SENDING";
+    case SENT:
+      return "SENT";
+    case NEGOTIATION_TIMED_OUT:
+      return "NEGOTIATION_TIMED_OUT";
+    case TIMED_OUT:
+      return "TIMED_OUT";
+    case CANCELLED:
+      return "CANCELLED";
+    case FINISHED_NEGOTIATION_ERROR:
+      return "FINISHED_NEGOTIATION_ERROR";
+    case FINISHED_ERROR:
+      return "FINISHED_ERROR";
+    case FINISHED_SUCCESS:
+      return "FINISHED_SUCCESS";
+    default:
+      LOG(DFATAL) << "Unknown state in OutboundCall: " << state;
+      return StringPrintf("UNKNOWN(%d)", state);
+  }
+}
+
+void OutboundCall::set_state(State new_state) {
+  std::lock_guard<simple_spinlock> l(lock_);
+  set_state_unlocked(new_state);
+}
+
+OutboundCall::State OutboundCall::state() const {
+  std::lock_guard<simple_spinlock> l(lock_);
+  return state_;
+}
+
+void OutboundCall::set_state_unlocked(State new_state) {
+  // Sanity check state transitions.
+  DVLOG(3) << "OutboundCall " << this << " (" << ToString() << ") switching from " <<
+    StateName(state_) << " to " << StateName(new_state);
+  switch (new_state) {
+    case ON_OUTBOUND_QUEUE:
+      DCHECK_EQ(state_, READY);
+      break;
+    case SENDING:
+      // Allow SENDING to be set idempotently so we don't have to specifically check
+      // whether the state is transitioning in the RPC code.
+      DCHECK(state_ == ON_OUTBOUND_QUEUE || state_ == SENDING);
+      break;
+    case SENT:
+      DCHECK_EQ(state_, SENDING);
+      break;
+    case NEGOTIATION_TIMED_OUT:
+      DCHECK(state_ == ON_OUTBOUND_QUEUE);
+      break;
+    case TIMED_OUT:
+      DCHECK(state_ == SENT || state_ == ON_OUTBOUND_QUEUE || state_ == SENDING);
+      break;
+    case CANCELLED:
+      DCHECK(state_ == READY || state_ == ON_OUTBOUND_QUEUE || state_ == SENT);
+      break;
+    case FINISHED_SUCCESS:
+      DCHECK_EQ(state_, SENT);
+      break;
+    default:
+      // No sanity checks for others.
+      break;
+  }
+
+  state_ = new_state;
+}
+
+void OutboundCall::Cancel() {
+  cancellation_requested_ = true;
+  // No lock needed as it's called from reactor thread
+  switch (state_) {
+    case READY:
+    case ON_OUTBOUND_QUEUE:
+    case SENT: {
+      SetCancelled();
+      break;
+    }
+    case SENDING:
+    case NEGOTIATION_TIMED_OUT:
+    case TIMED_OUT:
+    case CANCELLED:
+    case FINISHED_NEGOTIATION_ERROR:
+    case FINISHED_ERROR:
+    case FINISHED_SUCCESS:
+      break;
+  }
+}
+
+void OutboundCall::CallCallback() {
+  // Clear references to outbound sidecars before invoking callback.
+  sidecars_.clear();
+
+  int64_t start_cycles = CycleClock::Now();
+  {
+    SCOPED_WATCH_STACK(100);
+    callback_();
+    // Clear the callback, since it may be holding onto reference counts
+    // via bound parameters. We do this inside the timer because it's possible
+    // the user has naughty destructors that block, and we want to account for that
+    // time here if they happen to run on this thread.
+    callback_ = NULL;
+  }
+  int64_t end_cycles = CycleClock::Now();
+  int64_t wait_cycles = end_cycles - start_cycles;
+  if (PREDICT_FALSE(wait_cycles > FLAGS_rpc_callback_max_cycles)) {
+    double micros = static_cast<double>(wait_cycles) / base::CyclesPerSecond()
+      * kMicrosPerSecond;
+
+    LOG(WARNING) << "RPC callback for " << ToString() << " blocked reactor thread for "
+                 << micros << "us";
+  }
+}
+
+void OutboundCall::SetResponse(gscoped_ptr<CallResponse> resp) {
+  call_response_ = std::move(resp);
+  Slice r(call_response_->serialized_response());
+
+  if (call_response_->is_success()) {
+    // TODO: here we're deserializing the call response within the reactor thread,
+    // which isn't great, since it would block processing of other RPCs in parallel.
+    // Should look into a way to avoid this.
+    if (!response_->ParseFromArray(r.data(), r.size())) {
+      SetFailed(Status::IOError("invalid RPC response, missing fields",
+                                response_->InitializationErrorString()));
+      return;
+    }
+    set_state(FINISHED_SUCCESS);
+    CallCallback();
+  } else {
+    // Error
+    unique_ptr<ErrorStatusPB> err(new ErrorStatusPB());
+    if (!err->ParseFromArray(r.data(), r.size())) {
+      SetFailed(Status::IOError("Was an RPC error but could not parse error response",
+                                err->InitializationErrorString()));
+      return;
+    }
+    Status s = Status::RemoteError(err->message());
+    SetFailed(std::move(s), Phase::REMOTE_CALL, std::move(err));
+  }
+}
+
+void OutboundCall::SetQueued() {
+  set_state(ON_OUTBOUND_QUEUE);
+}
+
+void OutboundCall::SetSending() {
+  set_state(SENDING);
+}
+
+void OutboundCall::SetSent() {
+  set_state(SENT);
+
+  // This method is called in the reactor thread, so free the header buf,
+  // which was also allocated from this thread. tcmalloc's thread caching
+  // behavior is a lot more efficient if memory is freed from the same thread
+  // which allocated it -- this lets it keep to thread-local operations instead
+  // of taking a mutex to put memory back on the global freelist.
+  delete [] header_buf_.release();
+
+  // request_buf_ is also done being used here, but since it was allocated by
+  // the caller thread, we would rather let that thread free it whenever it
+  // deletes the RpcController.
+
+  // If cancellation was requested, it's now a good time to do the actual cancellation.
+  if (cancellation_requested()) {
+    SetCancelled();
+  }
+}
+
+void OutboundCall::SetFailed(Status status,
+                             Phase phase,
+                             unique_ptr<ErrorStatusPB> err_pb) {
+  DCHECK(!status.ok());
+  DCHECK(phase == Phase::CONNECTION_NEGOTIATION || phase == Phase::REMOTE_CALL);
+  {
+    std::lock_guard<simple_spinlock> l(lock_);
+    status_ = std::move(status);
+    error_pb_ = std::move(err_pb);
+    set_state_unlocked(phase == Phase::CONNECTION_NEGOTIATION
+        ? FINISHED_NEGOTIATION_ERROR
+        : FINISHED_ERROR);
+  }
+  CallCallback();
+}
+
+void OutboundCall::SetTimedOut(Phase phase) {
+  static const char* kErrMsgNegotiation =
+      "connection negotiation to $1 for RPC $0 timed out after $2 ($3)";
+  static const char* kErrMsgCall = "$0 RPC to $1 timed out after $2 ($3)";
+  DCHECK(phase == Phase::CONNECTION_NEGOTIATION || phase == Phase::REMOTE_CALL);
+
+  // We have to fetch timeout outside the lock to avoid a lock
+  // order inversion between this class and RpcController.
+  const MonoDelta timeout = controller_->timeout();
+  {
+    std::lock_guard<simple_spinlock> l(lock_);
+    status_ = Status::TimedOut(
+        Substitute((phase == Phase::REMOTE_CALL) ? kErrMsgCall : kErrMsgNegotiation,
+                   remote_method_.method_name(),
+                   conn_id_.remote().ToString(),
+                   timeout.ToString(),
+                   StateName(state_)));
+    set_state_unlocked((phase == Phase::REMOTE_CALL) ? TIMED_OUT : NEGOTIATION_TIMED_OUT);
+  }
+  CallCallback();
+}
+
+void OutboundCall::SetCancelled() {
+  DCHECK(!IsFinished());
+  {
+    std::lock_guard<simple_spinlock> l(lock_);
+    status_ = Status::Aborted(
+        Substitute("$0 RPC to $1 is cancelled in state $2",
+                   remote_method_.method_name(),
+                   conn_id_.remote().ToString(),
+                   StateName(state_)));
+    set_state_unlocked(CANCELLED);
+  }
+  CallCallback();
+}
+
+bool OutboundCall::IsTimedOut() const {
+  std::lock_guard<simple_spinlock> l(lock_);
+  switch (state_) {
+    case NEGOTIATION_TIMED_OUT:       // fall-through
+    case TIMED_OUT:
+      return true;
+    default:
+      return false;
+  }
+}
+
+bool OutboundCall::IsCancelled() const {
+  std::lock_guard<simple_spinlock> l(lock_);
+  return state_ == CANCELLED;
+}
+
+bool OutboundCall::IsNegotiationError() const {
+  std::lock_guard<simple_spinlock> l(lock_);
+  switch (state_) {
+    case FINISHED_NEGOTIATION_ERROR:  // fall-through
+    case NEGOTIATION_TIMED_OUT:
+      return true;
+    default:
+      return false;
+  }
+}
+
+bool OutboundCall::IsFinished() const {
+  std::lock_guard<simple_spinlock> l(lock_);
+  switch (state_) {
+    case READY:
+    case SENDING:
+    case ON_OUTBOUND_QUEUE:
+    case SENT:
+      return false;
+    case NEGOTIATION_TIMED_OUT:
+    case TIMED_OUT:
+    case CANCELLED:
+    case FINISHED_NEGOTIATION_ERROR:
+    case FINISHED_ERROR:
+    case FINISHED_SUCCESS:
+      return true;
+    default:
+      LOG(FATAL) << "Unknown call state: " << state_;
+      return false;
+  }
+}
+
+string OutboundCall::ToString() const {
+  return Substitute("RPC call $0 -> $1", remote_method_.ToString(), conn_id_.ToString());
+}
+
+void OutboundCall::DumpPB(const DumpRunningRpcsRequestPB& req,
+                          RpcCallInProgressPB* resp) {
+  std::lock_guard<simple_spinlock> l(lock_);
+  resp->mutable_header()->CopyFrom(header_);
+  resp->set_micros_elapsed((MonoTime::Now() - start_time_).ToMicroseconds());
+
+  switch (state_) {
+    case READY:
+      // Don't bother setting a state for "READY" since we don't expose a call
+      // until it's at least on the queue of a connection.
+      break;
+    case ON_OUTBOUND_QUEUE:
+      resp->set_state(RpcCallInProgressPB::ON_OUTBOUND_QUEUE);
+      break;
+    case SENDING:
+      resp->set_state(RpcCallInProgressPB::SENDING);
+      break;
+    case SENT:
+      resp->set_state(RpcCallInProgressPB::SENT);
+      break;
+    case NEGOTIATION_TIMED_OUT:
+      resp->set_state(RpcCallInProgressPB::NEGOTIATION_TIMED_OUT);
+      break;
+    case TIMED_OUT:
+      resp->set_state(RpcCallInProgressPB::TIMED_OUT);
+      break;
+    case CANCELLED:
+      resp->set_state(RpcCallInProgressPB::CANCELLED);
+      break;
+    case FINISHED_NEGOTIATION_ERROR:
+      resp->set_state(RpcCallInProgressPB::FINISHED_NEGOTIATION_ERROR);
+      break;
+    case FINISHED_ERROR:
+      resp->set_state(RpcCallInProgressPB::FINISHED_ERROR);
+      break;
+    case FINISHED_SUCCESS:
+      resp->set_state(RpcCallInProgressPB::FINISHED_SUCCESS);
+      break;
+  }
+}
+
+///
+/// CallResponse
+///
+
+CallResponse::CallResponse()
+ : parsed_(false) {
+}
+
+Status CallResponse::GetSidecar(int idx, Slice* sidecar) const {
+  DCHECK(parsed_);
+  if (idx < 0 || idx >= header_.sidecar_offsets_size()) {
+    return Status::InvalidArgument(strings::Substitute(
+        "Index $0 does not reference a valid sidecar", idx));
+  }
+  *sidecar = sidecar_slices_[idx];
+  return Status::OK();
+}
+
+Status CallResponse::ParseFrom(gscoped_ptr<InboundTransfer> transfer) {
+  CHECK(!parsed_);
+  RETURN_NOT_OK(serialization::ParseMessage(transfer->data(), &header_,
+                                            &serialized_response_));
+
+  // Use information from header to extract the payload slices.
+  RETURN_NOT_OK(RpcSidecar::ParseSidecars(header_.sidecar_offsets(),
+          serialized_response_, sidecar_slices_));
+
+  if (header_.sidecar_offsets_size() > 0) {
+    serialized_response_ =
+        Slice(serialized_response_.data(), header_.sidecar_offsets(0));
+  }
+
+  transfer_.swap(transfer);
+  parsed_ = true;
+  return Status::OK();
+}
+
+} // namespace rpc
+} // namespace kudu
diff --git a/be/src/kudu/rpc/outbound_call.h b/be/src/kudu/rpc/outbound_call.h
new file mode 100644
index 0000000..8d43891
--- /dev/null
+++ b/be/src/kudu/rpc/outbound_call.h
@@ -0,0 +1,348 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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.
+#ifndef KUDU_RPC_CLIENT_CALL_H
+#define KUDU_RPC_CLIENT_CALL_H
+
+#include <cstddef>
+#include <cstdint>
+#include <memory>
+#include <ostream>
+#include <set>
+#include <string>
+#include <vector>
+
+#include <gflags/gflags_declare.h>
+#include <glog/logging.h>
+#include <gtest/gtest_prod.h>
+
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/macros.h"
+#include "kudu/rpc/connection_id.h"
+#include "kudu/rpc/constants.h"
+#include "kudu/rpc/remote_method.h"
+#include "kudu/rpc/response_callback.h"
+#include "kudu/rpc/rpc_header.pb.h"
+#include "kudu/rpc/transfer.h"
+#include "kudu/util/faststring.h"
+#include "kudu/util/locks.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/slice.h"
+#include "kudu/util/status.h"
+
+DECLARE_int32(rpc_inject_cancellation_state);
+
+namespace google {
+namespace protobuf {
+class Message;
+} // namespace protobuf
+} // namespace google
+
+namespace kudu {
+namespace rpc {
+
+class CallResponse;
+class DumpRunningRpcsRequestPB;
+class RpcCallInProgressPB;
+class RpcController;
+class RpcSidecar;
+
+// Tracks the status of a call on the client side.
+//
+// This is an internal-facing class -- clients interact with the
+// RpcController class.
+//
+// This is allocated by the Proxy when a call is first created,
+// then passed to the reactor thread to send on the wire. It's typically
+// kept using a shared_ptr because a call may terminate in any number
+// of different threads, making it tricky to enforce single ownership.
+class OutboundCall {
+ public:
+
+  // Phases of an outbound RPC. Making an outbound RPC might involve establishing
+  // a connection to the remote server first, and the actual call is made only
+  // once the connection to the server is established.
+  enum class Phase {
+    // The phase of connection negotiation between the caller and the callee.
+    CONNECTION_NEGOTIATION,
+
+    // The phase of sending a call over already established connection.
+    REMOTE_CALL,
+  };
+
+  OutboundCall(const ConnectionId& conn_id, const RemoteMethod& remote_method,
+               google::protobuf::Message* response_storage,
+               RpcController* controller, ResponseCallback callback);
+
+  ~OutboundCall();
+
+  // Serialize the given request PB into this call's internal storage, and assume
+  // ownership of any sidecars that should accompany this request.
+  //
+  // Because the request data is fully serialized by this call, 'req' may be subsequently
+  // mutated with no ill effects.
+  void SetRequestPayload(const google::protobuf::Message& req,
+      std::vector<std::unique_ptr<RpcSidecar>>&& sidecars);
+
+  // Assign the call ID for this call. This is called from the reactor
+  // thread once a connection has been assigned. Must only be called once.
+  void set_call_id(int32_t call_id) {
+    DCHECK_EQ(header_.call_id(), kInvalidCallId) << "Already has a call ID";
+    header_.set_call_id(call_id);
+  }
+
+  // Serialize the call for the wire. Requires that SetRequestPayload()
+  // is called first. This is called from the Reactor thread.
+  // Returns the number of slices in the serialized call.
+  size_t SerializeTo(TransferPayload* slices);
+
+  // Mark in the call that cancellation has been requested. If the call hasn't yet
+  // started sending or has finished sending the RPC request but is waiting for a
+  // response, cancel the RPC right away. Otherwise, wait until the RPC has finished
+  // sending before cancelling it. If the call is finished, it's a no-op.
+  // REQUIRES: must be called from the reactor thread.
+  void Cancel();
+
+  // Callback after the call has been put on the outbound connection queue.
+  void SetQueued();
+
+  // Update the call state to show that the request has started being sent
+  // on the socket.
+  void SetSending();
+
+  // Update the call state to show that the request has been sent.
+  void SetSent();
+
+  // Mark the call as failed. This also triggers the callback to notify
+  // the caller. If the call failed due to a remote error, then err_pb
+  // should be set to the error returned by the remote server.
+  void SetFailed(Status status,
+                 Phase phase = Phase::REMOTE_CALL,
+                 std::unique_ptr<ErrorStatusPB> err_pb = nullptr);
+
+  // Mark the call as timed out. This also triggers the callback to notify
+  // the caller.
+  void SetTimedOut(Phase phase);
+  bool IsTimedOut() const;
+
+  bool IsNegotiationError() const;
+
+  bool IsCancelled() const;
+
+  // Is the call finished?
+  bool IsFinished() const;
+
+  // Fill in the call response.
+  void SetResponse(gscoped_ptr<CallResponse> resp);
+
+  const std::set<RpcFeatureFlag>& required_rpc_features() const {
+    return required_rpc_features_;
+  }
+
+  std::string ToString() const;
+
+  void DumpPB(const DumpRunningRpcsRequestPB& req, RpcCallInProgressPB* resp);
+
+  ////////////////////////////////////////////////////////////
+  // Getters
+  ////////////////////////////////////////////////////////////
+
+  const ConnectionId& conn_id() const { return conn_id_; }
+  const RemoteMethod& remote_method() const { return remote_method_; }
+  const ResponseCallback &callback() const { return callback_; }
+  RpcController* controller() { return controller_; }
+  const RpcController* controller() const { return controller_; }
+
+  // Return true if a call ID has been assigned to this call.
+  bool call_id_assigned() const {
+    return header_.call_id() != kInvalidCallId;
+  }
+
+  int32_t call_id() const {
+    DCHECK(call_id_assigned());
+    return header_.call_id();
+  }
+
+  // Returns true if cancellation has been requested. Must be called from
+  // reactor thread.
+  bool cancellation_requested() const {
+    return cancellation_requested_;
+  }
+
+  // Test function which returns true if a cancellation request should be injected
+  // at the current state.
+  bool ShouldInjectCancellation() const {
+    return FLAGS_rpc_inject_cancellation_state != -1 &&
+        FLAGS_rpc_inject_cancellation_state == state();
+  }
+
+ private:
+  friend class RpcController;
+  FRIEND_TEST(TestRpc, TestCancellation);
+
+  // Various states the call propagates through.
+  // NB: if adding another state, be sure to update OutboundCall::IsFinished()
+  // and OutboundCall::StateName(State state) as well.
+  enum State {
+    READY = 0,
+    ON_OUTBOUND_QUEUE,
+    SENDING,
+    SENT,
+    NEGOTIATION_TIMED_OUT,
+    TIMED_OUT,
+    CANCELLED,
+    FINISHED_NEGOTIATION_ERROR,
+    FINISHED_ERROR,
+    FINISHED_SUCCESS
+  };
+
+  static std::string StateName(State state);
+
+  // Mark the call as cancelled. This also invokes the callback to notify the caller.
+  void SetCancelled();
+
+  void set_state(State new_state);
+  State state() const;
+
+  // Same as set_state, but requires that the caller already holds
+  // lock_
+  void set_state_unlocked(State new_state);
+
+  // return current status
+  Status status() const;
+
+  // Time when the call was first initiatied.
+  MonoTime start_time_;
+
+  // Return the error protobuf, if a remote error occurred.
+  // This will only be non-NULL if status().IsRemoteError().
+  const ErrorStatusPB* error_pb() const;
+
+  // Lock for state_ status_, error_pb_ fields, since they
+  // may be mutated by the reactor thread while the client thread
+  // reads them.
+  mutable simple_spinlock lock_;
+  State state_;
+  Status status_;
+  std::unique_ptr<ErrorStatusPB> error_pb_;
+
+  // Call the user-provided callback. Note that entries in 'sidecars_' are cleared
+  // prior to invoking the callback so the client can assume that the call doesn't
+  // hold references to outbound sidecars.
+  void CallCallback();
+
+  // The RPC header.
+  // Parts of this (eg the call ID) are only assigned once this call has been
+  // passed to the reactor thread and assigned a connection.
+  RequestHeader header_;
+
+  // The remote method being called.
+  RemoteMethod remote_method_;
+
+  // RPC-system features required to send this call.
+  std::set<RpcFeatureFlag> required_rpc_features_;
+
+  const ConnectionId conn_id_;
+  ResponseCallback callback_;
+  RpcController* controller_;
+
+  // Pointer for the protobuf where the response should be written.
+  google::protobuf::Message* response_;
+
+  // Buffers for storing segments of the wire-format request.
+  faststring header_buf_;
+  faststring request_buf_;
+
+  // Once a response has been received for this call, contains that response.
+  // Otherwise NULL.
+  gscoped_ptr<CallResponse> call_response_;
+
+  // All sidecars to be sent with this call.
+  std::vector<std::unique_ptr<RpcSidecar>> sidecars_;
+
+  // Total size in bytes of all sidecars in 'sidecars_'. Set in SetRequestPayload().
+  // This cannot exceed TransferLimits::kMaxTotalSidecarBytes.
+  int32_t sidecar_byte_size_ = -1;
+
+  // True if cancellation was requested on this call.
+  bool cancellation_requested_;
+
+  DISALLOW_COPY_AND_ASSIGN(OutboundCall);
+};
+
+// A response to a call, on the client side.
+// Upon receiving a response, this is allocated in the reactor thread and filled
+// into the OutboundCall instance via OutboundCall::SetResponse.
+//
+// This may either be a success or error response.
+//
+// This class takes care of separating out the distinct payload slices sent
+// over.
+class CallResponse {
+ public:
+  CallResponse();
+
+  // Parse the response received from a call. This must be called before any
+  // other methods on this object.
+  Status ParseFrom(gscoped_ptr<InboundTransfer> transfer);
+
+  // Return true if the call succeeded.
+  bool is_success() const {
+    DCHECK(parsed_);
+    return !header_.is_error();
+  }
+
+  // Return the call ID that this response is related to.
+  int32_t call_id() const {
+    DCHECK(parsed_);
+    return header_.call_id();
+  }
+
+  // Return the serialized response data. This is just the response "body" --
+  // either a serialized ErrorStatusPB, or the serialized user response protobuf.
+  const Slice &serialized_response() const {
+    DCHECK(parsed_);
+    return serialized_response_;
+  }
+
+  // See RpcController::GetSidecar()
+  Status GetSidecar(int idx, Slice* sidecar) const;
+
+ private:
+  // True once ParseFrom() is called.
+  bool parsed_;
+
+  // The parsed header.
+  ResponseHeader header_;
+
+  // The slice of data for the encoded protobuf response.
+  // This slice refers to memory allocated by transfer_
+  Slice serialized_response_;
+
+  // Slices of data for rpc sidecars. They point into memory owned by transfer_.
+  Slice sidecar_slices_[TransferLimits::kMaxSidecars];
+
+  // The incoming transfer data - retained because serialized_response_
+  // and sidecar_slices_ refer into its data.
+  gscoped_ptr<InboundTransfer> transfer_;
+
+  DISALLOW_COPY_AND_ASSIGN(CallResponse);
+};
+
+} // namespace rpc
+} // namespace kudu
+
+#endif
diff --git a/be/src/kudu/rpc/periodic-test.cc b/be/src/kudu/rpc/periodic-test.cc
new file mode 100644
index 0000000..cd793bd
--- /dev/null
+++ b/be/src/kudu/rpc/periodic-test.cc
@@ -0,0 +1,295 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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.
+
+#include <atomic>
+#include <cstdint>
+#include <memory>
+#include <ostream>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include <glog/logging.h>
+#include <gtest/gtest.h>
+
+#include "kudu/rpc/messenger.h"
+#include "kudu/rpc/periodic.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/scoped_cleanup.h"
+#include "kudu/util/stopwatch.h"
+#include "kudu/util/test_macros.h"
+#include "kudu/util/test_util.h"
+
+using std::atomic;
+using std::shared_ptr;
+using std::vector;
+
+namespace kudu {
+namespace rpc {
+
+class PeriodicTimerTest : public KuduTest {
+ public:
+  PeriodicTimerTest()
+      : period_ms_(200) {}
+
+ protected:
+  const int64_t period_ms_;
+};
+
+class JitteredPeriodicTimerTest : public PeriodicTimerTest,
+                                  public ::testing::WithParamInterface<double> {
+ public:
+  // In TSAN builds it takes a long time to de-schedule a thread. Also,
+  // the actual time that thread spends sleeping in SleepFor() scenarios
+  // might be much longer than requested. Setting the task period to be long
+  // enough allows for more stable behavior of the test, so no flakiness
+  // is observed even under substantial load. Otherwise it would be necessary
+  // to introduce additional logic to verify that the actual timings satisfy
+  // the implicit constraints of the test scenarios below.
+  JitteredPeriodicTimerTest()
+      : counter_(0) {
+  }
+
+  virtual void SetUp() override {
+    PeriodicTimerTest::SetUp();
+
+    MessengerBuilder builder("test");
+    ASSERT_OK(builder.Build(&messenger_));
+
+    timer_ = PeriodicTimer::Create(messenger_,
+                                   [&] { counter_++; },
+                                   MonoDelta::FromMilliseconds(period_ms_),
+                                   GetOptions());
+  }
+
+  virtual void TearDown() override {
+    // Ensure that the reactor threads are fully quiesced (and thus no timer
+    // callbacks are running) by the time 'counter_' is destroyed.
+    messenger_->Shutdown();
+
+    KuduTest::TearDown();
+  }
+
+ protected:
+
+  virtual PeriodicTimer::Options GetOptions() {
+    PeriodicTimer::Options opts;
+    opts.jitter_pct = GetParam();
+    return opts;
+  }
+
+  atomic<int64_t> counter_;
+  shared_ptr<Messenger> messenger_;
+  shared_ptr<PeriodicTimer> timer_;
+};
+
+INSTANTIATE_TEST_CASE_P(AllJitterModes,
+                        JitteredPeriodicTimerTest,
+                        ::testing::Values(0.0, 0.25));
+
+TEST_P(JitteredPeriodicTimerTest, TestStartStop) {
+  // Before the timer starts, the counter's value should not change.
+  SleepFor(MonoDelta::FromMilliseconds(period_ms_ * 2));
+  ASSERT_EQ(0, counter_);
+
+  // Once started, it should increase (exactly how much depends on load and the
+  // underlying OS scheduler).
+  timer_->Start();
+  SleepFor(MonoDelta::FromMilliseconds(period_ms_ * 2));
+  ASSERT_EVENTUALLY([&]{
+    ASSERT_GT(counter_, 0);
+  });
+
+  // After stopping the timer, the value should either remain the same or
+  // increment once (if Stop() raced with a scheduled task).
+  timer_->Stop();
+  int64_t v = counter_;
+  messenger_->Shutdown();
+  ASSERT_TRUE(counter_ == v ||
+              counter_ == v + 1);
+}
+
+TEST_P(JitteredPeriodicTimerTest, TestReset) {
+  timer_->Start();
+  MonoTime start_time = MonoTime::Now();
+
+  // Loop for a little while, resetting the timer's period over and over. As a
+  // result, the timer should never fire.
+  while (true) {
+    MonoTime now = MonoTime::Now();
+    if (now - start_time > MonoDelta::FromMilliseconds(period_ms_ * 5)) {
+      break;
+    }
+    timer_->Snooze();
+    ASSERT_EQ(0, counter_);
+    SleepFor(MonoDelta::FromMilliseconds(1));
+  }
+}
+
+TEST_P(JitteredPeriodicTimerTest, TestResetWithDelta) {
+  timer_->Start();
+  timer_->Snooze(MonoDelta::FromMilliseconds(period_ms_ * 5));
+
+  // One period later, the counter still hasn't incremented...
+  SleepFor(MonoDelta::FromMilliseconds(period_ms_));
+  ASSERT_EQ(0, counter_);
+
+  // ...but it will increment eventually.
+  ASSERT_EVENTUALLY([&](){
+    ASSERT_GT(counter_, 0);
+  });
+}
+
+TEST_P(JitteredPeriodicTimerTest, TestStartWithDelta) {
+  timer_->Start(MonoDelta::FromMilliseconds(period_ms_ * 5));
+
+  // One period later, the counter still hasn't incremented...
+  SleepFor(MonoDelta::FromMilliseconds(period_ms_));
+  ASSERT_EQ(0, counter_);
+
+  // ...but it will increment eventually.
+  ASSERT_EVENTUALLY([&](){
+    ASSERT_GT(counter_, 0);
+  });
+}
+
+TEST_F(PeriodicTimerTest, TestCallbackRestartsTimer) {
+  const int64_t kPeriods = 10;
+
+  shared_ptr<Messenger> messenger;
+  ASSERT_OK(MessengerBuilder("test").Build(&messenger));
+
+  // Create a timer that restarts itself from within its functor.
+  PeriodicTimer::Options opts;
+  opts.jitter_pct = 0.0; // don't need jittering
+  shared_ptr<PeriodicTimer> timer = PeriodicTimer::Create(
+      messenger,
+      [&] {
+        timer->Stop();
+        timer->Start();
+      },
+      MonoDelta::FromMilliseconds(period_ms_),
+      std::move(opts));
+
+  // Run the timer for a fixed amount of time.
+  timer->Start();
+  SleepFor(MonoDelta::FromMilliseconds(period_ms_ * kPeriods));
+  timer->Stop();
+
+  // Although the timer is restarted by its functor, its overall period should
+  // remain more or less the same (since the period expired just as the functor
+  // ran). As such, we should see no more than three callbacks per period:
+  // one to start scheduling the callback loop, one when it fires, and one more
+  // after it has been replaced by a new callback loop.
+  ASSERT_LE(timer->NumCallbacksForTests(), kPeriods * 3);
+}
+
+class JitteredOneShotPeriodicTimerTest : public JitteredPeriodicTimerTest {
+ protected:
+  virtual PeriodicTimer::Options GetOptions() override {
+    PeriodicTimer::Options opts;
+    opts.jitter_pct = GetParam();
+    opts.one_shot = true;
+    return opts;
+  }
+};
+
+INSTANTIATE_TEST_CASE_P(AllJitterModes,
+                        JitteredOneShotPeriodicTimerTest,
+                        ::testing::Values(0.0, 0.25));
+
+TEST_P(JitteredOneShotPeriodicTimerTest, TestBasics) {
+  // Kick off the one-shot timer a few times.
+  for (int i = 0; i < 3; i++) {
+    ASSERT_EQ(i, counter_);
+
+    // Eventually the task will run.
+    timer_->Start();
+    ASSERT_EVENTUALLY([&](){
+      ASSERT_EQ(i + 1, counter_);
+    });
+
+    // Even if we explicitly wait another few periods, the counter value
+    // shouldn't change.
+    SleepFor(MonoDelta::FromMilliseconds(period_ms_ * 2));
+    ASSERT_EQ(i + 1, counter_);
+  }
+}
+
+TEST_F(PeriodicTimerTest, TestCallbackRestartsOneShotTimer) {
+  atomic<int64_t> counter(0);
+  shared_ptr<Messenger> messenger;
+  ASSERT_OK(MessengerBuilder("test")
+            .Build(&messenger));
+
+  // Create a timer that restarts itself from within its functor.
+  PeriodicTimer::Options opts;
+  opts.jitter_pct = 0.0; // don't need jittering
+  opts.one_shot = true;
+  shared_ptr<PeriodicTimer> timer = PeriodicTimer::Create(
+      messenger,
+      [&] {
+        counter++;
+        timer->Start();
+      },
+      MonoDelta::FromMilliseconds(period_ms_),
+      std::move(opts));
+
+  // Because the timer restarts itself every time the functor runs, we
+  // should see the counter value increase with each period.
+  timer->Start();
+  ASSERT_EVENTUALLY([&](){
+    ASSERT_GE(counter, 5);
+  });
+
+  // Ensure that the reactor threads are fully quiesced (and thus no timer
+  // callbacks are running) by the time 'counter' is destroyed.
+  messenger->Shutdown();
+}
+
+TEST_F(PeriodicTimerTest, TestPerformance) {
+  const int kNumTimers = 1000;
+  shared_ptr<Messenger> messenger;
+  ASSERT_OK(MessengerBuilder("test")
+            .set_num_reactors(1)
+            .Build(&messenger));
+  SCOPED_CLEANUP({ messenger->Shutdown(); });
+
+  vector<shared_ptr<PeriodicTimer>> timers;
+  for (int i = 0; i < kNumTimers; i++) {
+    timers.emplace_back(PeriodicTimer::Create(
+        messenger,
+        [&] {}, // No-op.
+        MonoDelta::FromMilliseconds(10)));
+    timers.back()->Start();
+  }
+
+  Stopwatch sw(Stopwatch::ALL_THREADS);
+  sw.start();
+  SleepFor(MonoDelta::FromSeconds(1));
+  sw.stop();
+  LOG(INFO) << "User CPU for running " << kNumTimers << " timers for 1 second: "
+            << sw.elapsed().user_cpu_seconds() << "s";
+
+  for (auto& t : timers) {
+    t->Stop();
+  }
+
+}
+
+} // namespace rpc
+} // namespace kudu
diff --git a/be/src/kudu/rpc/periodic.cc b/be/src/kudu/rpc/periodic.cc
new file mode 100644
index 0000000..9ab9ce9
--- /dev/null
+++ b/be/src/kudu/rpc/periodic.cc
@@ -0,0 +1,219 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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.
+
+#include "kudu/rpc/periodic.h"
+
+#include <algorithm>
+#include <memory>
+#include <mutex>
+
+#include <boost/function.hpp>
+#include <glog/logging.h>
+
+#include "kudu/rpc/messenger.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/random.h"
+#include "kudu/util/random_util.h"
+#include "kudu/util/status.h"
+
+using std::shared_ptr;
+using std::weak_ptr;
+
+namespace kudu {
+namespace rpc {
+
+PeriodicTimer::Options::Options()
+    : jitter_pct(0.25),
+      one_shot(false) {
+}
+
+shared_ptr<PeriodicTimer> PeriodicTimer::Create(
+    shared_ptr<Messenger> messenger,
+    RunTaskFunctor functor,
+    MonoDelta period,
+    Options options) {
+  return PeriodicTimer::make_shared(
+      std::move(messenger), std::move(functor), period, options);
+}
+
+PeriodicTimer::PeriodicTimer(
+    shared_ptr<Messenger> messenger,
+    RunTaskFunctor functor,
+    MonoDelta period,
+    Options options)
+    : messenger_(std::move(messenger)),
+      functor_(std::move(functor)),
+      period_(period),
+      options_(options),
+      rng_(GetRandomSeed32()),
+      current_callback_generation_(0),
+      num_callbacks_for_tests_(0),
+      started_(false) {
+  DCHECK_GE(options_.jitter_pct, 0);
+  DCHECK_LE(options_.jitter_pct, 1);
+}
+
+PeriodicTimer::~PeriodicTimer() {
+  Stop();
+}
+
+void PeriodicTimer::Start(boost::optional<MonoDelta> next_task_delta) {
+  std::unique_lock<simple_spinlock> l(lock_);
+  if (!started_) {
+    started_ = true;
+    SnoozeUnlocked(std::move(next_task_delta));
+    int new_callback_generation = ++current_callback_generation_;
+
+    // Invoke Callback() with the lock released.
+    l.unlock();
+    Callback(new_callback_generation);
+  }
+}
+
+void PeriodicTimer::Stop() {
+  std::lock_guard<simple_spinlock> l(lock_);
+  StopUnlocked();
+}
+
+void PeriodicTimer::StopUnlocked() {
+  DCHECK(lock_.is_locked());
+  started_ = false;
+}
+
+void PeriodicTimer::Snooze(boost::optional<MonoDelta> next_task_delta) {
+  std::lock_guard<simple_spinlock> l(lock_);
+  SnoozeUnlocked(std::move(next_task_delta));
+}
+
+void PeriodicTimer::SnoozeUnlocked(boost::optional<MonoDelta> next_task_delta) {
+  DCHECK(lock_.is_locked());
+  if (!started_) {
+    return;
+  }
+
+  if (!next_task_delta) {
+    // Given jitter percentage J and period P, this yields a delay somewhere
+    // between (1-J)*P and (1+J)*P.
+    next_task_delta = MonoDelta::FromMilliseconds(
+        GetMinimumPeriod().ToMilliseconds() +
+        rng_.NextDoubleFraction() *
+        options_.jitter_pct *
+        (2 * period_.ToMilliseconds()));
+  }
+  next_task_time_ = MonoTime::Now() + *next_task_delta;
+}
+
+bool PeriodicTimer::started() const {
+  std::lock_guard<simple_spinlock> l(lock_);
+  return started_;
+}
+
+MonoDelta PeriodicTimer::GetMinimumPeriod() {
+  // Given jitter percentage J and period P, this returns (1-J)*P, which is
+  // the lowest possible jittered value.
+  return MonoDelta::FromMilliseconds((1.0 - options_.jitter_pct) *
+                                     period_.ToMilliseconds());
+}
+
+int64_t PeriodicTimer::NumCallbacksForTests() const {
+  std::lock_guard<simple_spinlock> l(lock_);
+  return num_callbacks_for_tests_;
+}
+
+void PeriodicTimer::Callback(int64_t my_callback_generation) {
+  // To simplify the implementation, a timer may have only one outstanding
+  // callback scheduled at a time. This means that once the callback is
+  // scheduled, the timer's task cannot run any earlier than whenever the
+  // callback runs. Thus, the delay used when scheduling the callback dictates
+  // the lowest possible value of 'next_task_delta' that Snooze() can honor.
+  //
+  // If the callback's delay is very low, Snooze() can honor a low
+  // 'next_task_delta', but the callback will run often and burn more CPU
+  // cycles. If the delay is very high, the timer will be more efficient but
+  // the granularity for 'next_task_delta' will rise accordingly.
+  //
+  // As a "happy medium" we use GetMinimumPeriod() as the delay. This ensures
+  // that a no-arg Snooze() on a jittered timer will always be honored, and as
+  // long as the caller passes a value of at least GetMinimumPeriod() to
+  // Snooze(), that too will be honored.
+  MonoDelta delay = GetMinimumPeriod();
+  bool run_task = false;
+  {
+    std::lock_guard<simple_spinlock> l(lock_);
+    num_callbacks_for_tests_++;
+
+    // If the timer was stopped, exit.
+    if (!started_) {
+      return;
+    }
+
+    // If there's a new callback loop in town, exit.
+    //
+    // We could check again just before calling Messenger::ScheduleOnReactor()
+    // (in case someone else restarted the timer while the functor ran, or in
+    // case the functor itself restarted the timer), but there's no real reason
+    // to do so: the very next iteration of this callback loop will wind up here
+    // and exit.
+    if (current_callback_generation_ > my_callback_generation) {
+      return;
+    }
+
+    MonoTime now = MonoTime::Now();
+    if (now < next_task_time_) {
+      // It's not yet time to run the task. Reduce the scheduled delay if
+      // enough time has elapsed, but don't increase it.
+      delay = std::min(delay, next_task_time_ - now);
+    } else {
+      // It's time to run the task. Although the next task time is reset now,
+      // it may be reset again by virtue of running the task itself.
+      run_task = true;
+
+      if (options_.one_shot) {
+        // Stop the timer first, in case the task wants to restart it.
+        StopUnlocked();
+      }
+    }
+  }
+
+  if (run_task) {
+    functor_();
+
+    if (options_.one_shot) {
+      // The task was run; exit the loop. Even if the task restarted the timer,
+      // that will have started a new callback loop, so exiting here is always
+      // the correct thing to do.
+      return;
+    }
+    Snooze();
+  }
+
+  // Capture a weak_ptr reference into the submitted functor so that we can
+  // safely handle the functor outliving its timer.
+  weak_ptr<PeriodicTimer> w = shared_from_this();
+  messenger_->ScheduleOnReactor([w, my_callback_generation](const Status& s) {
+    if (!s.ok()) {
+      // The reactor was shut down.
+      return;
+    }
+    if (auto timer = w.lock()) {
+      timer->Callback(my_callback_generation);
+    }
+  }, delay);
+}
+
+} // namespace rpc
+} // namespace kudu
diff --git a/be/src/kudu/rpc/periodic.h b/be/src/kudu/rpc/periodic.h
new file mode 100644
index 0000000..592cbd5
--- /dev/null
+++ b/be/src/kudu/rpc/periodic.h
@@ -0,0 +1,215 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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.
+
+#pragma once
+
+#include <cstdint>
+#include <functional>
+#include <memory>
+
+#include <boost/optional/optional.hpp>
+#include <gtest/gtest_prod.h>
+
+#include "kudu/gutil/macros.h"
+#include "kudu/util/locks.h"
+#include "kudu/util/make_shared.h"
+#include "kudu/util/monotime.h"
+#include "kudu/util/random.h"
+
+namespace kudu {
+namespace rpc {
+
+class Messenger;
+
+// Repeatedly runs a task on a fixed period.
+//
+// PeriodicTimer's periodicity is maintained via tail recursive calls to
+// Messenger::ScheduleOnReactor(). Every time the scheduled callback is
+// invoked, it checks the current time, updates some internal bookkeeping,
+// runs the user's task if the time is right, and makes another call to
+// Messenger::ScheduleOnReactor() to run itself again in the future. This
+// looping behavior is called a "callback loop".
+//
+// Every time Stop() and then Start() (or just Start(), if this is the first
+// such call) are invoked, PeriodicTimer will kick off a new callback loop. If
+// there was an old loop, it remains intact until its scheduled callback runs,
+// at which point it will detect that a new loop was created and exit.
+//
+// The use of Messenger::ScheduleOnReactor() is merely for convenience;
+// PeriodicTimer could also be built on libev, a hashed wheel timer, o
+// something equivalent.
+//
+// PeriodicTimers have shared ownership, but that's largely an implementation
+// detail to support asynchronous stopping. Users can treat them as exclusively
+// owned (though care must be taken when writing the task functor; see Stop()
+// for more details).
+//
+// TODO(adar): eventually we should build directly on libev as it supports
+// timer cancelation, which would allow us to implement synchronous Stop(), use
+// exclusive ownership, and remove the restriction that the delta passed
+// into Snooze() be greater than GetMinimumPeriod().
+class PeriodicTimer : public std::enable_shared_from_this<PeriodicTimer>,
+                      public enable_make_shared<PeriodicTimer> {
+ public:
+  typedef std::function<void(void)> RunTaskFunctor;
+
+  struct Options {
+    Options();
+
+    // Defines the percentage of the period that will be jittered up or down
+    // randomly. Together with the period, the periodicity of the timer will
+    // vary between (1-J)*P and (1+J)*P.
+    //
+    // Must be between 0 and 1.
+    //
+    // If not set, defaults to 0.25.
+    double jitter_pct;
+
+    // The timer will automatically stop after running the user's task.
+    //
+    // Just as with a normal timer, Snooze() will postpone the running of the
+    // task, and Stop() will cancel the task outright. Unlike a normal timer,
+    // both operations will no-op if the timer has already fired.
+    //
+    // If not set, defaults to false.
+    bool one_shot;
+  };
+
+  // Creates a new PeriodicTimer.
+  //
+  // A ref is taken on 'messenger', which is used for scheduling callbacks.
+  //
+  // 'functor' defines the user's task and is owned for the lifetime of the
+  // PeriodicTimer. The task will run on the messenger's reactor threads so it
+  // should do very little work (i.e. no I/O).
+  //
+  // 'period' defines the period between tasks.
+  //
+  // 'options' allows additional (optional) customization of the timer.
+  static std::shared_ptr<PeriodicTimer> Create(
+      std::shared_ptr<Messenger> messenger,
+      RunTaskFunctor functor,
+      MonoDelta period,
+      Options options = {});
+
+  ~PeriodicTimer();
+
+  // Starts the timer.
+  //
+  // The timer's task will run in accordance with the period and jitter mode
+  // provided during timer construction.
+  //
+  // If 'next_task_delta' is set, it is used verbatim as the delay for the very
+  // first task, with the configured period and jitter mode only applying to
+  // subsequent tasks.
+  //
+  // Does nothing if the timer was already started.
+  void Start(boost::optional<MonoDelta> next_task_delta = boost::none);
+
+  // Snoozes the timer for one period.
+  //
+  // If 'next_task_delta' is set, it is used verbatim as the delay for the next
+  // task. Subsequent tasks will revert to the timer's regular period. The
+  // value of 'next_task_delta' must be greater than GetMinimumPeriod();
+  // otherwise the task is not guaranteed to run in a timely manner.
+  //
+  // Note: Snooze() is not additive. That is, if called at time X and again at
+  // time X + P/2, the timer is snoozed until X+P/2+P, not X+2P.
+  //
+  // Does nothing if the timer is stopped.
+  void Snooze(boost::optional<MonoDelta> next_task_delta = boost::none);
+
+  // Stops the timer.
+  //
+  // Stopping is asynchronous; that is, it is still possible for the task to
+  // run after Stop() returns. Because of this, the task's functor should be
+  // written to do nothing if objects it depends on have been destroyed.
+  //
+  // Does nothing if the timer is already stopped.
+  void Stop();
+
+  // Returns true iff the timer has been started.
+  bool started() const;
+
+ protected:
+  PeriodicTimer(std::shared_ptr<Messenger> messenger,
+                RunTaskFunctor functor,
+                MonoDelta period,
+                Options options);
+
+ private:
+  FRIEND_TEST(PeriodicTimerTest, TestCallbackRestartsTimer);
+  // Calculate the minimum period for the timer, which varies depending on
+  // 'jitter_pct_' and the output of the PRNG.
+  MonoDelta GetMinimumPeriod();
+
+  // Called by Messenger::ScheduleOnReactor when the timer fires.
+  // 'my_callback_generation' is the callback generation assigned to this loop
+  // when it was constructed.
+  void Callback(int64_t my_callback_generation);
+
+  // Like Stop() but must be called with 'lock_' held.
+  void StopUnlocked();
+
+  // Like Snooze() but must be called with 'lock_' held.
+  void SnoozeUnlocked(boost::optional<MonoDelta> next_task_delta = boost::none);
+
+  // Returns the number of times that Callback() has been called by this timer.
+  //
+  // Should only be used for tests!
+  int64_t NumCallbacksForTests() const;
+
+  // Schedules invocations of Callback() in the future.
+  std::shared_ptr<Messenger> messenger_;
+
+  // User-defined task functor.
+  RunTaskFunctor functor_;
+
+  // User-specified task period.
+  const MonoDelta period_;
+
+  // User-specified options.
+  const Options options_;
+
+  // Protects all mutable state below.
+  mutable simple_spinlock lock_;
+
+  // PRNG used when generating jitter.
+  Random rng_;
+
+  // The next time at which the task's functor should be run.
+  MonoTime next_task_time_;
+
+  // The most recent callback generation.
+  //
+  // When started, a callback loop is assigned a generation, which it remembers
+  // for its entire lifespan. If 'current_callback_generation_' exceeds the
+  // loop's assigned generation, that means another loop has been created and
+  // the (now old) loop should exit.
+  int64_t current_callback_generation_;
+
+  // The number of times that Callback() has been invoked.
+  int64_t num_callbacks_for_tests_;
+
+  // Whether the timer is running or not.
+  bool started_;
+
+  DISALLOW_COPY_AND_ASSIGN(PeriodicTimer);
+};
+
+} // namespace rpc
+} // namespace kudu
diff --git a/be/src/kudu/rpc/protoc-gen-krpc.cc b/be/src/kudu/rpc/protoc-gen-krpc.cc
new file mode 100644
index 0000000..cc25f67
--- /dev/null
+++ b/be/src/kudu/rpc/protoc-gen-krpc.cc
@@ -0,0 +1,691 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you 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.
+
+////////////////////////////////////////////////////////////////////////////////
+// Example usage:
+// protoc --plugin=protoc-gen-krpc --krpc_out . --proto_path . <file>.proto
+////////////////////////////////////////////////////////////////////////////////
+
+#include <cstddef>
+#include <map>
+#include <memory>
+#include <set>
+#include <string>
+#include <vector>
+
+#include <boost/optional/optional.hpp>
+#include <glog/logging.h>
+#include <google/protobuf/compiler/code_generator.h>
+#include <google/protobuf/compiler/plugin.h>
+#include <google/protobuf/descriptor.h>
+#include <google/protobuf/descriptor.pb.h>
+#include <google/protobuf/io/printer.h>
+#include <google/protobuf/io/zero_copy_stream.h>
+
+#include "kudu/gutil/gscoped_ptr.h"
+#include "kudu/gutil/port.h"
+#include "kudu/gutil/strings/join.h"
+#include "kudu/gutil/strings/numbers.h"
+#include "kudu/gutil/strings/split.h"
+#include "kudu/gutil/strings/stringpiece.h"
+#include "kudu/gutil/strings/strip.h"
+#include "kudu/gutil/strings/substitute.h"
+#include "kudu/gutil/strings/util.h"
+#include "kudu/rpc/rpc_header.pb.h"
+#include "kudu/util/status.h"
+#include "kudu/util/string_case.h"
+
+using boost::optional;
+using google::protobuf::FileDescriptor;
+using google::protobuf::io::Printer;
+using google::protobuf::MethodDescriptor;
+using google::protobuf::ServiceDescriptor;
+using std::map;
+using std::shared_ptr;
+using std::set;
+using std::string;
+using std::vector;
+
+namespace kudu {
+namespace rpc {
+
+namespace {
+
+// Return the name of the authorization method specified for this
+// RPC method, or boost::none if none is specified.
+//
+// This handles fallback to the service-wide default.
+optional<string> GetAuthzMethod(const MethodDescriptor& method) {
+  if (method.options().HasExtension(authz_method)) {
+    return method.options().GetExtension(authz_method);
+  }
+  if (method.service()->options().HasExtension(default_authz_method)) {
+    return method.service()->options().GetExtension(default_authz_method);
+  }
+  return boost::none;
+}
+
+} // anonymous namespace
+
+class Substituter {
+ public:
+  virtual ~Substituter() {}
+  virtual void InitSubstitutionMap(map<string, string> *map) const = 0;
+};
+
+// NameInfo contains information about the output names.
+class FileSubstitutions : public Substituter {
+ public:
+  static const std::string kProtoExtension;
+
+  Status Init(const FileDescriptor *file) {
+    const string& path = file->name();
+    map_["path"] = path;
+
+    // Initialize path_
+    // If path = /foo/bar/baz_stuff.proto, path_ = /foo/bar/baz_stuff
+    if (!TryStripSuffixString(path, kProtoExtension, &path_no_extension_)) {
+      return Status::InvalidArgument("file name " + path +
+                                     " did not end in " + kProtoExtension);
+    }
+    map_["path_no_extension"] = path_no_extension_;
+
+    // If path = /foo/bar/baz_stuff.proto, base_ = baz_stuff
+    string base;
+    GetBaseName(path_no_extension_, &base);
+    map_["base"] = base;
+
+    // If path = /foo/bar/baz_stuff.proto, camel_case_ = BazStuff
+    string camel_case;
+    SnakeToCamelCase(base, &camel_case);
+    map_["camel_case"] = camel_case;
+
+    // If path = /foo/bar/baz_stuff.proto, upper_case_ = BAZ_STUFF
+    string upper_case;
+    ToUpperCase(base, &upper_case);
+    map_["upper_case"] = upper_case;
+
+    map_["open_namespace"] = GenerateOpenNamespace(file->package());
+    map_["close_namespace"] = GenerateCloseNamespace(file->package());
+
+    return Status::OK();
+  }
+
+  virtual void InitSubstitutionMap(map<string, string> *map) const OVERRIDE {
+    typedef std::map<string, string>::value_type kv_pair;
+    for (const kv_pair &pair : map_) {
+      (*map)[pair.first] = pair.second;
+    }
+  }
+
+  std::string service_header() const {
+    return path_no_extension_ + ".service.h";
+  }
+
+  std::string service() const {
+    return path_no_extension_ + ".service.cc";
+  }
+
+  std::string proxy_header() const {
+    return path_no_extension_ + ".proxy.h";
+  }
+
+  std::string proxy() const {
+    return path_no_extension_ + ".proxy.cc";
+  }
+
+ private:
+  // Extract the last filename component.
+  static void GetBaseName(const string &path,
+                          string *base) {
+    size_t last_slash = path.find_last_of('/');
+    if (last_slash != string::npos) {
+      *base = path.substr(last_slash + 1);
+    } else {
+      *base = path;
+    }
+  }
+
+  static string GenerateOpenNamespace(const string &str) {
+    vector<string> components = strings::Split(str, ".");
+    string out;
+    for (const string &c : components) {
+      out.append("namespace ").append(c).append(" {\n");
+    }
+    return out;
+  }
+
+  static string GenerateCloseNamespace(const string &str) {
+    vector<string> components = strings::Split(str, ".");
+    string out;
+    for (auto c = components.crbegin(); c != components.crend(); c++) {
+      out.append("} // namespace ").append(*c).append("\n");
+    }
+    return out;
+  }
+
+  std::string path_no_extension_;
+  map<string, string> map_;
+};
+
+const std::string FileSubstitutions::kProtoExtension(".proto");
+
+class MethodSubstitutions : public Substituter {
+ public:
+  explicit MethodSubstitutions(const MethodDescriptor *method)
+    : method_(method) {
+  }
+
+  virtual void InitSubstitutionMap(map<string, string> *map) const OVERRIDE {
+
+    (*map)["rpc_name"] = method_->name();
+    (*map)["rpc_full_name"] = method_->full_name();
+    (*map)["rpc_full_name_plainchars"] =
+        StringReplace(method_->full_name(), ".", "_", true);
+    (*map)["request"] =
+        ReplaceNamespaceDelimiters(
+            StripNamespaceIfPossible(method_->service()->full_name(),
+                                     method_->input_type()->full_name()));
+    (*map)["response"] =
+        ReplaceNamespaceDelimiters(
+            StripNamespaceIfPossible(method_->service()->full_name(),
+                                     method_->output_type()->full_name()));
+    (*map)["metric_enum_key"] = strings::Substitute("kMetricIndex$0", method_->name());
+    bool track_result = static_cast<bool>(method_->options().GetExtension(track_rpc_result));
+    (*map)["track_result"] = track_result ? " true" : "false";
+    (*map)["authz_method"] = GetAuthzMethod(*method_).get_value_or("AuthorizeAllowAll");
+  }
+
+  // Strips the package from method arguments if they are in the same package as
+  // the service, otherwise leaves them so that we can have fully qualified
+  // namespaces for method arguments.
+  static std::string StripNamespaceIfPossible(const std::string& service_full_name,
+                                              const std::string& arg_full_name) {
+    StringPiece service_package(service_full_name);
+    if (!service_package.contains(".")) {
+      return arg_full_name;
+    }
+    // remove the service name so that we are left with only the package, including
+    // the last '.' so that we account for different packages with the same prefix.
+    service_package.remove_suffix(service_package.length() -
+                                  service_package.find_last_of(".") - 1);
+
+    StringPiece argfqn(arg_full_name);
+    if (argfqn.starts_with(service_package)) {
+      argfqn.remove_prefix(argfqn.find_last_of(".") + 1);
+    }
+    return argfqn.ToString();
+  }
+
+  static std::string ReplaceNamespaceDelimiters(const std::string& arg_full_name) {
+    return JoinStrings(strings::Split(arg_full_name, "."), "::");
+  }
+
+ private:
+  const MethodDescriptor *method_;
+};
+
+class ServiceSubstitutions : public Substituter {
+ public:
+  explicit ServiceSubstitutions(const ServiceDescriptor *service)
+    : service_(service)
+  {}
+
+  virtual void InitSubstitutionMap(map<string, string> *map) const OVERRIDE {
+    (*map)["service_name"] = service_->name();
+    (*map)["full_service_name"] = service_->full_name();
+    (*map)["service_method_count"] = SimpleItoa(service_->method_count());
+
+    // TODO: upgrade to protobuf 2.5.x and attach service comments
+    // to the generated service classes using the SourceLocation API.
+  }
+
+ private:
+  const ServiceDescriptor *service_;
+};
+
+
+class SubstitutionContext {
+ public:
+  // Takes ownership of the substituter
+  void Push(const Substituter *sub) {
+    subs_.push_back(shared_ptr<const Substituter>(sub));
+  }
+
+  void PushMethod(const MethodDescriptor *method) {
+    Push(new MethodSubstitutions(method));
+  }
+
+  void PushService(const ServiceDescriptor *service) {
+    Push(new ServiceSubstitutions(service));
+  }
+
+  void Pop() {
+    CHECK(!subs_.empty());
+    subs_.pop_back();
+  }
+
+  void InitSubstitutionMap(map<string, string> *subs) const {
+    for (const shared_ptr<const Substituter> &sub : subs_) {
+      sub->InitSubstitutionMap(subs);
+    }
+  }
+
+ private:
+  vector<shared_ptr<const Substituter> > subs_;
+};
+
+
+
+class CodeGenerator : public ::google::protobuf::compiler::CodeGenerator {
+ public:
+  CodeGenerator() { }
+
+  ~CodeGenerator() { }
+
+  bool Generate(const google::protobuf::FileDescriptor *file,
+        const std::string &/* parameter */,
+        google::protobuf::compiler::GeneratorContext *gen_context,
+        std::string *error) const OVERRIDE {
+    auto name_info = new FileSubstitutions();
+    Status ret = name_info->Init(file);
+    if (!ret.ok()) {
+      *error = "name_info.Init failed: " + ret.ToString();
+      return false;
+    }
+
+    SubstitutionContext subs;
+    subs.Push(name_info);
+
+    gscoped_ptr<google::protobuf::io::ZeroCopyOutputStream> ih_output(
+        gen_context->Open(name_info->service_header()));
+    Printer ih_printer(ih_output.get(), '$');
+    GenerateServiceIfHeader(&ih_printer, &subs, file);
+
+    gscoped_ptr<google::protobuf::io::ZeroCopyOutputStream> i_output(
+        gen_context->Open(name_info->service()));
+    Printer i_printer(i_output.get(), '$');
+    GenerateServiceIf(&i_printer, &subs, file);
+
+    gscoped_ptr<google::protobuf::io::ZeroCopyOutputStream> ph_output(
+        gen_context->Open(name_info->proxy_header()));
+    Printer ph_printer(ph_output.get(), '$');
+    GenerateProxyHeader(&ph_printer, &subs, file);
+
+    gscoped_ptr<google::protobuf::io::ZeroCopyOutputStream> p_output(
+        gen_context->Open(name_info->proxy()));
+    Printer p_printer(p_output.get(), '$');
+    GenerateProxy(&p_printer, &subs, file);
+
+    return true;
+  }
+
+ private:
+  void Print(Printer *printer,
+             const SubstitutionContext &sub,
+             const char *text) const {
+    map<string, string> subs;
+    sub.InitSubstitutionMap(&subs);
+    printer->Print(subs, text);
+  }
+
+  void GenerateServiceIfHeader(Printer *printer,
+                               SubstitutionContext *subs,
+                               const FileDescriptor *file) const {
+    Print(printer, *subs,
+      "// THIS FILE IS AUTOGENERATED FROM $path$\n"
+      "\n"
+      "#ifndef KUDU_RPC_$upper_case$_SERVICE_IF_DOT_H\n"
+      "#define KUDU_RPC_$upper_case$_SERVICE_IF_DOT_H\n"
+      "\n"
+      "#include <string>\n"
+      "\n"
+      "#include \"kudu/gutil/ref_counted.h\"\n"
+      "#include \"kudu/rpc/service_if.h\"\n"
+      "\n"
+      "namespace google {\n"
+      "namespace protobuf {\n"
+      "class Message;\n"
+      "} // namespace protobuf\n"
+      "} // namespace google\n"
+      "\n"
+      "namespace kudu {\n"
+      "class MetricEntity;\n"
... 92305 lines suppressed ...