diff --git a/CHANGELOG.md b/CHANGELOG.md
index 9ba11e011..23aeb417e 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -11,6 +11,8 @@ Versioning](http://semver.org/spec/v2.0.0.html) except to the first release.
 ### Added
 
 - Support queue 1.2.0 (#177)
+- ConnectionObserver interface for knowing changes of connections in
+  ConnectionPool (#178)
 
 ### Changed
 
diff --git a/connection_pool/connection_pool.go b/connection_pool/connection_pool.go
index e0c862386..f54baf54d 100644
--- a/connection_pool/connection_pool.go
+++ b/connection_pool/connection_pool.go
@@ -32,10 +32,29 @@ var (
 	ErrNoHealthyInstance = errors.New("can't find healthy instance in pool")
 )
 
+// ConnectionObserver provides an observer semantics for components interested
+// in knowing changes of connections in a ConnectionPool.
+type ConnectionObserver interface {
+	// Opened is called when a new connection has been opened. It has not yet
+	// been added to the pool of active connections. It allows for a client to
+	// initialize the connection before using it in a pool.
+	//
+	// The client code may cancel adding a connection to the pool. The client
+	// need to return an error from the Opened call for that. In this case
+	// the pool will close connection and will try to reopen it later.
+	Opened(conn *tarantool.Connection, role Role) error
+	// Closed is called after a connection with a role removed from a list
+	// of active connections. If a role of the connection is updated, a
+	// pool calls Closed() + Opened().
+	Closed(conn *tarantool.Connection, role Role) error
+}
+
 /*
 Additional options (configurable via ConnectWithOpts):
 
 - CheckTimeout - time interval to check for connection timeout and try to switch connection.
+
+- ConnectionObserver - an observer for connection updates.
 */
 type OptsPool struct {
 	// timeout for timer to reopen connections
@@ -43,6 +62,8 @@ type OptsPool struct {
 	// to relocate connection between subpools
 	// if ro/rw role has been updated
 	CheckTimeout time.Duration
+	// ConnectionObserver provides an ability to observe connection updates.
+	ConnectionObserver ConnectionObserver
 }
 
 /*
@@ -69,15 +90,21 @@ type ConnectionPool struct {
 	connOpts tarantool.Opts
 	opts     OptsPool
 
-	notify     chan tarantool.ConnEvent
 	state      state
-	control    chan struct{}
+	done       chan struct{}
 	roPool     *RoundRobinStrategy
 	rwPool     *RoundRobinStrategy
 	anyPool    *RoundRobinStrategy
 	poolsMutex sync.RWMutex
 }
 
+type connState struct {
+	addr   string
+	notify chan tarantool.ConnEvent
+	conn   *tarantool.Connection
+	role   Role
+}
+
 // ConnectWithOpts creates pool for instances with addresses addrs
 // with options opts.
 func ConnectWithOpts(addrs []string, connOpts tarantool.Opts, opts OptsPool) (connPool *ConnectionPool, err error) {
@@ -88,9 +115,6 @@ func ConnectWithOpts(addrs []string, connOpts tarantool.Opts, opts OptsPool) (co
 		return nil, ErrWrongCheckTimeout
 	}
 
-	notify := make(chan tarantool.ConnEvent, 10*len(addrs)) // x10 to accept disconnected and closed event (with a margin)
-	connOpts.Notify = notify
-
 	size := len(addrs)
 	rwPool := NewEmptyRoundRobin(size)
 	roPool := NewEmptyRoundRobin(size)
@@ -100,9 +124,8 @@ func ConnectWithOpts(addrs []string, connOpts tarantool.Opts, opts OptsPool) (co
 		addrs:    make([]string, 0, len(addrs)),
 		connOpts: connOpts,
 		opts:     opts,
-		notify:   notify,
 		state:    unknownState,
-		control:  make(chan struct{}),
+		done:     make(chan struct{}),
 		rwPool:   rwPool,
 		roPool:   roPool,
 		anyPool:  anyPool,
@@ -116,15 +139,21 @@ func ConnectWithOpts(addrs []string, connOpts tarantool.Opts, opts OptsPool) (co
 		}
 	}
 
-	somebodyAlive := connPool.fillPools()
+	states, somebodyAlive := connPool.fillPools()
 	if !somebodyAlive {
 		connPool.state.set(closedState)
 		connPool.closeImpl()
+		for _, s := range states {
+			close(s.notify)
+		}
 		return nil, ErrNoConnection
 	}
 
 	connPool.state.set(connectedState)
-	go connPool.checker()
+
+	for _, s := range states {
+		go connPool.checker(s)
+	}
 
 	return connPool, nil
 }
@@ -172,8 +201,6 @@ func (connPool *ConnectionPool) ConfiguredTimeout(mode Mode) (time.Duration, err
 }
 
 func (connPool *ConnectionPool) closeImpl() []error {
-	close(connPool.control)
-
 	errs := make([]error, 0, len(connPool.addrs))
 
 	for _, addr := range connPool.addrs {
@@ -181,11 +208,18 @@ func (connPool *ConnectionPool) closeImpl() []error {
 			if err := conn.Close(); err != nil {
 				errs = append(errs, err)
 			}
+
+			role := UnknownRole
+			if conn := connPool.rwPool.DeleteConnByAddr(addr); conn != nil {
+				role = MasterRole
+			} else if conn := connPool.roPool.DeleteConnByAddr(addr); conn != nil {
+				role = ReplicaRole
+			}
+			connPool.connectionObserverClosed(conn, role)
 		}
-		connPool.rwPool.DeleteConnByAddr(addr)
-		connPool.roPool.DeleteConnByAddr(addr)
 	}
 
+	close(connPool.done)
 	return errs
 }
 
@@ -665,19 +699,17 @@ func (connPool *ConnectionPool) getConnectionFromPool(addr string) (*tarantool.C
 	return connPool.anyPool.GetConnByAddr(addr), UnknownRole
 }
 
-func (connPool *ConnectionPool) deleteConnectionFromPool(addr string) {
-	_ = connPool.anyPool.DeleteConnByAddr(addr)
-	conn := connPool.rwPool.DeleteConnByAddr(addr)
-	if conn == nil {
+func (connPool *ConnectionPool) deleteConnection(addr string) {
+	if conn := connPool.anyPool.DeleteConnByAddr(addr); conn != nil {
+		if conn := connPool.rwPool.DeleteConnByAddr(addr); conn != nil {
+			return
+		}
 		connPool.roPool.DeleteConnByAddr(addr)
 	}
 }
 
-func (connPool *ConnectionPool) setConnectionToPool(addr string, conn *tarantool.Connection) error {
-	role, err := connPool.getConnectionRole(conn)
-	if err != nil {
-		return err
-	}
+func (connPool *ConnectionPool) addConnection(addr string,
+	conn *tarantool.Connection, role Role) {
 
 	connPool.anyPool.AddConn(addr, conn)
 
@@ -687,85 +719,228 @@ func (connPool *ConnectionPool) setConnectionToPool(addr string, conn *tarantool
 	case ReplicaRole:
 		connPool.roPool.AddConn(addr, conn)
 	}
+}
 
-	return nil
+func (connPool *ConnectionPool) connectionObserverOpened(conn *tarantool.Connection,
+	role Role) bool {
+	var err error
+	if connPool.opts.ConnectionObserver != nil {
+		err = connPool.opts.ConnectionObserver.Opened(conn, role)
+	}
+
+	if err != nil {
+		addr := conn.Addr()
+		log.Printf("tarantool: storing connection to %s canceled: %s\n", addr, err)
+		return false
+	}
+	return true
 }
 
-func (connPool *ConnectionPool) refreshConnection(addr string) {
-	if conn, oldRole := connPool.getConnectionFromPool(addr); conn != nil {
-		if !conn.ClosedNow() {
-			curRole, _ := connPool.getConnectionRole(conn)
-			if oldRole != curRole {
-				connPool.deleteConnectionFromPool(addr)
-				err := connPool.setConnectionToPool(addr, conn)
-				if err != nil {
-					conn.Close()
-					log.Printf("tarantool: storing connection to %s failed: %s\n", addr, err.Error())
-				}
-			}
+func (connPool *ConnectionPool) connectionObserverClosed(conn *tarantool.Connection,
+	role Role) {
+	var err error
+	if connPool.opts.ConnectionObserver != nil {
+		err = connPool.opts.ConnectionObserver.Closed(conn, role)
+	}
+
+	if err != nil {
+		addr := conn.Addr()
+		log.Printf("tarantool: removing connection to %s by user failed: %s\n", addr, err)
+	}
+}
+
+func (connPool *ConnectionPool) fillPools() ([]connState, bool) {
+	states := make([]connState, len(connPool.addrs))
+	somebodyAlive := false
+
+	// It is called before checker() goroutines and before closeImpl() may be
+	// called so we don't expect concurrency issues here.
+	for i, addr := range connPool.addrs {
+		states[i] = connState{
+			addr:   addr,
+			notify: make(chan tarantool.ConnEvent, 10),
+			conn:   nil,
+			role:   UnknownRole,
 		}
-	} else {
-		conn, _ := tarantool.Connect(addr, connPool.connOpts)
-		if conn != nil {
-			err := connPool.setConnectionToPool(addr, conn)
+		connOpts := connPool.connOpts
+		connOpts.Notify = states[i].notify
+
+		conn, err := tarantool.Connect(addr, connOpts)
+		if err != nil {
+			log.Printf("tarantool: connect to %s failed: %s\n", addr, err.Error())
+		} else if conn != nil {
+			role, err := connPool.getConnectionRole(conn)
 			if err != nil {
 				conn.Close()
-				log.Printf("tarantool: storing connection to %s failed: %s\n", addr, err.Error())
+				log.Printf("tarantool: storing connection to %s failed: %s\n", addr, err)
+				continue
+			}
+
+			if connPool.connectionObserverOpened(conn, role) {
+				connPool.addConnection(addr, conn, role)
+
+				if conn.ConnectedNow() {
+					states[i].conn = conn
+					states[i].role = role
+					somebodyAlive = true
+				} else {
+					conn.Close()
+					connPool.connectionObserverClosed(conn, role)
+				}
+			} else {
+				conn.Close()
+				connPool.connectionObserverClosed(conn, role)
 			}
 		}
 	}
+
+	return states, somebodyAlive
 }
 
-func (connPool *ConnectionPool) checker() {
-	timer := time.NewTicker(connPool.opts.CheckTimeout)
-	defer timer.Stop()
+func (pool *ConnectionPool) updateConnection(s connState) connState {
+	pool.poolsMutex.Lock()
 
-	for connPool.state.get() != closedState {
-		select {
-		case <-connPool.control:
-			return
-		case e := <-connPool.notify:
-			connPool.poolsMutex.Lock()
-			if connPool.state.get() == connectedState && e.Conn.ClosedNow() {
-				connPool.deleteConnectionFromPool(e.Conn.Addr())
+	if pool.state.get() != connectedState {
+		pool.poolsMutex.Unlock()
+		return s
+	}
+
+	if role, err := pool.getConnectionRole(s.conn); err == nil {
+		if s.role != role {
+			pool.deleteConnection(s.addr)
+			pool.poolsMutex.Unlock()
+
+			pool.connectionObserverClosed(s.conn, s.role)
+			opened := pool.connectionObserverOpened(s.conn, role)
+			if !opened {
+				s.conn.Close()
+				pool.connectionObserverClosed(s.conn, role)
+				s.conn = nil
+				s.role = UnknownRole
+				return s
 			}
-			connPool.poolsMutex.Unlock()
-		case <-timer.C:
-			connPool.poolsMutex.Lock()
-			if connPool.state.get() == connectedState {
-				for _, addr := range connPool.addrs {
-					// Reopen connection
-					// Relocate connection between subpools
-					// if ro/rw was updated
-					connPool.refreshConnection(addr)
-				}
+
+			pool.poolsMutex.Lock()
+			if pool.state.get() != connectedState {
+				pool.poolsMutex.Unlock()
+
+				s.conn.Close()
+				pool.connectionObserverClosed(s.conn, role)
+				s.conn = nil
+				s.role = UnknownRole
+				return s
 			}
-			connPool.poolsMutex.Unlock()
+
+			pool.addConnection(s.addr, s.conn, role)
+			s.role = role
 		}
 	}
+
+	pool.poolsMutex.Unlock()
+	return s
 }
 
-func (connPool *ConnectionPool) fillPools() bool {
-	somebodyAlive := false
+func (pool *ConnectionPool) tryConnect(s connState) connState {
+	pool.poolsMutex.Lock()
+
+	if pool.state.get() != connectedState {
+		pool.poolsMutex.Unlock()
+		return s
+	}
+
+	s.conn = nil
+	s.role = UnknownRole
+
+	connOpts := pool.connOpts
+	connOpts.Notify = s.notify
+	conn, _ := tarantool.Connect(s.addr, connOpts)
+	if conn != nil {
+		role, err := pool.getConnectionRole(conn)
+		pool.poolsMutex.Unlock()
 
-	// It is called before checker() goroutine and before closeImpl() may be
-	// called so we don't expect concurrency issues here.
-	for _, addr := range connPool.addrs {
-		conn, err := tarantool.Connect(addr, connPool.connOpts)
 		if err != nil {
-			log.Printf("tarantool: connect to %s failed: %s\n", addr, err.Error())
-		} else if conn != nil {
-			err = connPool.setConnectionToPool(addr, conn)
-			if err != nil {
-				conn.Close()
-				log.Printf("tarantool: storing connection to %s failed: %s\n", addr, err.Error())
-			} else if conn.ConnectedNow() {
-				somebodyAlive = true
-			}
+			conn.Close()
+			log.Printf("tarantool: storing connection to %s failed: %s\n", s.addr, err)
+			return s
 		}
+
+		opened := pool.connectionObserverOpened(conn, role)
+		if !opened {
+			conn.Close()
+			pool.connectionObserverClosed(conn, role)
+			return s
+		}
+
+		pool.poolsMutex.Lock()
+		if pool.state.get() != connectedState {
+			pool.poolsMutex.Unlock()
+			conn.Close()
+			pool.connectionObserverClosed(conn, role)
+			return s
+		}
+
+		pool.addConnection(s.addr, conn, role)
+		s.conn = conn
+		s.role = role
+	}
+
+	pool.poolsMutex.Unlock()
+	return s
+}
+
+func (pool *ConnectionPool) reconnect(s connState) connState {
+	pool.poolsMutex.Lock()
+
+	if pool.state.get() != connectedState {
+		pool.poolsMutex.Unlock()
+		return s
 	}
 
-	return somebodyAlive
+	pool.deleteConnection(s.addr)
+	pool.poolsMutex.Unlock()
+
+	pool.connectionObserverClosed(s.conn, s.role)
+	s.conn = nil
+	s.role = UnknownRole
+
+	return pool.tryConnect(s)
+}
+
+func (pool *ConnectionPool) checker(s connState) {
+	timer := time.NewTicker(pool.opts.CheckTimeout)
+	defer timer.Stop()
+
+	for {
+		select {
+		case <-pool.done:
+			close(s.notify)
+			return
+		case <-s.notify:
+			if s.conn != nil && s.conn.ClosedNow() {
+				pool.poolsMutex.Lock()
+				if pool.state.get() == connectedState {
+					pool.deleteConnection(s.addr)
+					pool.poolsMutex.Unlock()
+					pool.connectionObserverClosed(s.conn, s.role)
+					s.conn = nil
+					s.role = UnknownRole
+				} else {
+					pool.poolsMutex.Unlock()
+				}
+			}
+		case <-timer.C:
+			// Reopen connection
+			// Relocate connection between subpools
+			// if ro/rw was updated
+			if s.conn == nil {
+				s = pool.tryConnect(s)
+			} else if !s.conn.ClosedNow() {
+				s = pool.updateConnection(s)
+			} else {
+				s = pool.reconnect(s)
+			}
+		}
+	}
 }
 
 func (connPool *ConnectionPool) getNextConnection(mode Mode) (*tarantool.Connection, error) {
diff --git a/connection_pool/connection_pool_test.go b/connection_pool/connection_pool_test.go
index 326604b51..446789d98 100644
--- a/connection_pool/connection_pool_test.go
+++ b/connection_pool/connection_pool_test.go
@@ -6,6 +6,7 @@ import (
 	"os"
 	"reflect"
 	"strings"
+	"sync"
 	"testing"
 	"time"
 
@@ -233,6 +234,257 @@ func TestClose(t *testing.T) {
 	require.Nil(t, err)
 }
 
+type testObserver struct {
+	opened, closed int
+	errs           []error
+	mutex          sync.Mutex
+}
+
+func (tl *testObserver) addErr(err error) {
+	tl.errs = append(tl.errs, err)
+}
+
+func (tl *testObserver) Opened(conn *tarantool.Connection,
+	role connection_pool.Role) error {
+	tl.mutex.Lock()
+	defer tl.mutex.Unlock()
+
+	tl.opened++
+
+	if conn == nil {
+		tl.addErr(fmt.Errorf("opened conn == nil"))
+		return nil
+	}
+
+	// opened < 3 - initial open of connections
+	// opened >= 3 - reopen of connection after a role update
+	addr := conn.Addr()
+	if addr == servers[0] {
+		if tl.opened < 3 && role != connection_pool.MasterRole {
+			tl.addErr(fmt.Errorf("unexpected init role %d for addr %s", role, addr))
+		}
+		if tl.opened >= 3 && role != connection_pool.ReplicaRole {
+			tl.addErr(fmt.Errorf("unexpected updated role %d for addr %s", role, addr))
+		}
+	} else if addr == servers[1] {
+		if tl.opened >= 3 {
+			tl.addErr(fmt.Errorf("unexpected open for addr %s", addr))
+		}
+		if role != connection_pool.ReplicaRole {
+			tl.addErr(fmt.Errorf("unexpected role %d for addr %s", role, addr))
+		}
+	} else {
+		tl.addErr(fmt.Errorf("unexpected opened addr %s", addr))
+	}
+
+	return nil
+}
+
+func (tl *testObserver) Closed(conn *tarantool.Connection,
+	role connection_pool.Role) error {
+	tl.mutex.Lock()
+	defer tl.mutex.Unlock()
+
+	tl.closed++
+
+	if conn == nil {
+		tl.addErr(fmt.Errorf("removed conn == nil"))
+		return nil
+	}
+
+	addr := conn.Addr()
+	if tl.closed == 1 && addr == servers[0] {
+		// A first close is a role update.
+		if role != connection_pool.MasterRole {
+			tl.addErr(fmt.Errorf("unexpected removed role %d for addr %s", role, addr))
+		}
+		return nil
+	}
+
+	if addr == servers[0] || addr == servers[1] {
+		// Close.
+		if role != connection_pool.ReplicaRole {
+			tl.addErr(fmt.Errorf("unexpected removed role %d for addr %s", role, addr))
+		}
+	} else {
+		tl.addErr(fmt.Errorf("unexpected removed addr %s", addr))
+	}
+
+	return nil
+}
+
+func TestConnectionObserverOpenUpdateClose(t *testing.T) {
+	poolServers := []string{servers[0], servers[1]}
+	roles := []bool{false, true}
+
+	err := test_helpers.SetClusterRO(poolServers, connOpts, roles)
+	require.Nilf(t, err, "fail to set roles for cluster")
+
+	tl := &testObserver{}
+	poolOpts := connection_pool.OptsPool{
+		CheckTimeout:       100 * time.Microsecond,
+		ConnectionObserver: tl,
+	}
+	pool, err := connection_pool.ConnectWithOpts(poolServers, connOpts, poolOpts)
+	require.Nilf(t, err, "failed to connect")
+	require.NotNilf(t, pool, "conn is nil after Connect")
+
+	_, err = pool.Call17("box.cfg", []interface{}{map[string]bool{
+		"read_only": true,
+	}}, connection_pool.RW)
+	require.Nilf(t, err, "failed to make ro")
+
+	for i := 0; i < 100; i++ {
+		// Wait for read_only update, it should report about close connection
+		// with old role.
+		if tl.closed >= 1 {
+			break
+		}
+		time.Sleep(poolOpts.CheckTimeout)
+	}
+	require.Equalf(t, tl.closed, 1, "updated not reported as closed")
+	require.Equalf(t, tl.opened, 3, "updated not reported as opened")
+
+	pool.Close()
+
+	for i := 0; i < 100; i++ {
+		// Wait for close of all connections.
+		if tl.closed >= 3 {
+			break
+		}
+		time.Sleep(poolOpts.CheckTimeout)
+	}
+
+	for _, err := range tl.errs {
+		t.Errorf("Unexpected error: %s", err)
+	}
+	connected, err := pool.ConnectedNow(connection_pool.ANY)
+	require.Nilf(t, err, "failed to get connected state")
+	require.Falsef(t, connected, "connection pool still be connected")
+	require.Equalf(t, len(poolServers)+1, tl.opened, "unexpected opened count")
+	require.Equalf(t, len(poolServers)+1, tl.closed, "unexpected closed count")
+}
+
+type testAddErrorObserver struct {
+	connection_pool.ConnectionObserver
+	opened, closed int
+}
+
+func (tl *testAddErrorObserver) Opened(conn *tarantool.Connection,
+	role connection_pool.Role) error {
+	tl.opened++
+	return fmt.Errorf("any error")
+}
+
+func (tl *testAddErrorObserver) Closed(conn *tarantool.Connection,
+	role connection_pool.Role) error {
+	tl.closed++
+	return nil
+}
+
+func TestConnectionObserverAddError(t *testing.T) {
+	poolServers := []string{servers[0], servers[1]}
+
+	tl := &testAddErrorObserver{}
+	poolOpts := connection_pool.OptsPool{
+		CheckTimeout:       100 * time.Microsecond,
+		ConnectionObserver: tl,
+	}
+	connPool, err := connection_pool.ConnectWithOpts(poolServers, connOpts, poolOpts)
+	if err == nil {
+		defer connPool.Close()
+	}
+	require.NotNilf(t, err, "success to connect")
+	require.Equalf(t, 2, tl.opened, "unexpected opened count")
+	require.Equalf(t, 2, tl.closed, "unexpected closed count")
+}
+
+type testUpdateErrorObserver struct {
+	connection_pool.ConnectionObserver
+	opened, closed int
+	mutex          sync.Mutex
+}
+
+func (tl *testUpdateErrorObserver) Opened(conn *tarantool.Connection,
+	role connection_pool.Role) error {
+	tl.mutex.Lock()
+	defer tl.mutex.Unlock()
+
+	tl.opened++
+
+	if tl.closed != 0 {
+		// Don't add a connection into a pool again after it was deleted.
+		return fmt.Errorf("any error")
+	}
+	return nil
+}
+
+func (tl *testUpdateErrorObserver) Closed(conn *tarantool.Connection,
+	role connection_pool.Role) error {
+	tl.mutex.Lock()
+	defer tl.mutex.Unlock()
+
+	tl.closed++
+	return nil
+}
+
+func TestConnectionObserverUpdateError(t *testing.T) {
+	poolServers := []string{servers[0], servers[1]}
+	roles := []bool{false, false}
+
+	err := test_helpers.SetClusterRO(poolServers, connOpts, roles)
+	require.Nilf(t, err, "fail to set roles for cluster")
+
+	tl := &testUpdateErrorObserver{}
+	poolOpts := connection_pool.OptsPool{
+		CheckTimeout:       100 * time.Microsecond,
+		ConnectionObserver: tl,
+	}
+	connPool, err := connection_pool.ConnectWithOpts(poolServers, connOpts, poolOpts)
+	require.Nilf(t, err, "failed to connect")
+	require.NotNilf(t, connPool, "conn is nil after Connect")
+	defer connPool.Close()
+
+	connected, err := connPool.ConnectedNow(connection_pool.ANY)
+	require.Nilf(t, err, "failed to get ConnectedNow()")
+	require.Truef(t, connected, "should be connected")
+
+	for i := 0; i < len(poolServers); i++ {
+		_, err = connPool.Call17("box.cfg", []interface{}{map[string]bool{
+			"read_only": true,
+		}}, connection_pool.RW)
+		require.Nilf(t, err, "failed to make ro")
+	}
+
+	for i := 0; i < 100; i++ {
+		// Wait for updates done.
+		if tl.closed >= 4 {
+			break
+		}
+		time.Sleep(poolOpts.CheckTimeout)
+	}
+	connected, err = connPool.ConnectedNow(connection_pool.ANY)
+
+	require.Nilf(t, err, "failed to get ConnectedNow()")
+	require.Falsef(t, connected, "should not be any active connection")
+
+	connPool.Close()
+
+	for i := 0; i < 100; i++ {
+		// Wait for closed all.
+		if tl.closed != tl.opened {
+			break
+		}
+		time.Sleep(poolOpts.CheckTimeout)
+	}
+	connected, err = connPool.ConnectedNow(connection_pool.ANY)
+
+	require.Nilf(t, err, "failed to get ConnectedNow()")
+	require.Falsef(t, connected, "should be closed")
+	require.GreaterOrEqualf(t, tl.opened, tl.closed, "opened != closed")
+	require.Nilf(t, err, "failed to get ConnectedNow()")
+}
+
 func TestRequestOnClosed(t *testing.T) {
 	server1 := servers[0]
 	server2 := servers[1]