Skip to content

Campaign watch cancel causes adapter/in-process watch stream permanent blocking #20716

@Masorini

Description

@Masorini

Bug report criteria

What happened?

I have an etcd cluster with many C++ services connecting to it for leader election. Since the C++ client SDK (etcd-cpp-apiv3) does not provide an election API like the Go concurrency package, we implement leader election by calling the "/v3/election/campaign" gRPC method. This is implemented via adapter.WatchServerToWatchClient, which simulates a WatchClient and WatchServer interaction inside the server process, rather than through real gRPC socket connections.

When using adapter/in-process watch (WatchServerToWatchClient) to simulate client-server gRPC in-process, many business watches are multiplexed onto the same server-side stream. If a large number of watches are canceled simultaneously (e.g., due to failover or network flaps), the watch send/receive flow can enter a sustained blocking / deadlock-like state. New watches stop functioning correctly and etcd_debugging_mvcc_pending_events_total/ etcd_debugging_mvcc_slow_watcher_total spikes and remains high (observed >30 minutes without recovery).

Even after shutting down all campaign client processes, the blocking does not self-heal.

What did you expect to happen?

  1. Watch send/receive should not enter permanent blocking/deadlock when many cancels occur through "/v3/election/campaign".

  2. After the cancellation spike subsides, pending events should be drained and subsequent watches should resume normal operation.

How can we reproduce it (as minimally and precisely as possible)?

Original discovery was with C++ client (etcd-cpp-apiv3). The steps below are a minimal Go reproduction run against a local 3-node etcd (v3.5.13).

  1. Start a local 3-node etcd cluster (v3.5.13).
  2. Start client A (primary): issue N (e.g. 500) Campaign requests (different keys).
  3. Start client B (backup): issue N Campaign requests for the same keys; B will block on watch(waitDelete).
  4. Wait a few minutes then kill all B processes at once, producing many watch cancel events.
  5. Observe etcd_debugging_mvcc_pending_events_total and etcd_debugging_mvcc_slow_watcher_total metrics. After the spike, metrics do not recover; new watches stop receiving events. (Later, I initiated two separate campaign requests to simulate a leader and a follower. When I shut down the leader, the follower failed to be promoted to the new leader as expected.)

Increase N (500 → 800 → 1000) if needed to trigger reliably.

After reproduction, the metrics remain at a high level and do not decrease:

etcd_debugging_mvcc_pending_events_total 128
etcd_debugging_mvcc_slow_watcher_total 329

The client code I used for reproduction is as follows:

var prefix = flag.String("p", "default", "the config file")

func createEtcdClient() (*clientv3.Client, error) {
	cli, err := clientv3.New(clientv3.Config{
		Endpoints:            []string{"127.0.0.2:2379"},
		DialKeepAliveTime:    time.Second * 10,
		DialKeepAliveTimeout: time.Second * 5,
		Logger:               logx.ZapLogger(),
	})
	if err != nil {
		return nil, err
	}

	return cli, nil
}

func moreClient() {
	i := 0
	for i < 500 {
		pre := ""
		if i < 10 {
			pre = fmt.Sprintf("00%d", i)
		} else if i < 100 {
			pre = fmt.Sprintf("0%d", i)
		} else {
			pre = fmt.Sprintf("%d", i)
		}

		go func() {
			campaignMore(pre)
		}()

		i++
	}
	time.Sleep(time.Second * 300000)
}

func campaignMore(prefix string) {
	ctx := context.Background()
	cli, err := createEtcdClient()
	fmt.Println(cli, err)

	leaseResp, err := cli.Grant(ctx, 4)
	fmt.Println(leaseResp, err)
	cli.KeepAlive(ctx, leaseResp.ID)

	elector := v3electionpb.NewElectionClient(cli.ActiveConnection())

	req := v3electionpb.CampaignRequest{
		Name:  []byte("/config/murraylin_test/election_" + prefix),
		Lease: int64(leaseResp.ID),
		Value: []byte("test"),
	}
	var resp *v3electionpb.CampaignResponse
	retry := 0
	for {
		resp, err = elector.Campaign(ctx, &req)

		if err == nil {
			break
		}

		retry++
		if retry >= 3 {
			panic(err)
		}
	}
	fmt.Println(resp)

	resignReq := v3electionpb.ResignRequest{
		Leader: resp.Leader,
	}

	for {
		logx.Info("I am the leader now")
		time.Sleep(time.Second)
	}
	elector.Resign(ctx, &resignReq)
}

func main() {
	flag.Parse()
	if *prefix == "default" {
		moreClient()
	} else {
		campaignMore(*prefix)
	}
}

Anything else we need to know?

Line numbers are based on v3.5.13.

  1. wgs.run issues many watchCancelRequest (/etcd/client/v3/watch.go:692).

  2. Server sws.recvLoop receives cancels and enqueues control responses into server.sws.ctrlStream (buffer len = 16). Writes block when queue is full. (/etcd/server/api/v3rpc/watch.go:348)

  3. Server sws.sendLoop reads from ctrlStream/event queue and calls gRPC Send() to the downstream. If downstream Send() blocks (client not consuming), sendLoop blocks and cannot drain ctrlStream. (/etcd/server/api/v3rpc/watch.go:478)

  4. recvLoop then blocks on writing, and client wgs.run waits on wgs.respc for responses and cannot progress (/etcd/client/v3/watch.go:579).

  5. The combination forms a cyclic dependency resulting in sustained blocking; ctrlStream cannot be cleared and pending events accumulate indefinitely.

Tested on v3.6.4, and the issue still persists.
This issue might be related to #18879 or share similar characteristics.

The following diagram illustrates the deadlock between the in-process watch client and server.
Image

Etcd version (please run commands below)

$ etcd --version
# paste output here
etcd Version: 3.5.13
Git SHA: c9063a0dc
Go Version: go1.20.10
Go OS/Arch: darwin/arm64

$ etcdctl version
# paste output here
etcdctl version: 3.5.13
API version: 3.5

Etcd configuration (command line flags or environment variables)

paste your configuration here

Etcd debug information (please run commands below, feel free to obfuscate the IP address or FQDN in the output)

$ etcdctl member list -w table
# paste output here

$ etcdctl --endpoints=<member list> endpoint status -w table
# paste output here

# goroutine stack info
goroutine profile: total 5464
800 @ 0x102b82b18 0x102b467a8 0x102b81cd0 0x102bc6ca8 0x102bc7f5c 0x102bc7f4d 0x102d11378 0x102d1f8c4 0x103443278 0x1034445e8 0x102db4670 0x102bbe6e0 0x102fa9b08 0x102fa9add 0x102faa1d4 0x102ff82b8 0x10306648c 0x103065d84 0x102b8b2b4
#	0x102b81ccf	internal/poll.runtime_pollWait+0x9f						/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/runtime/netpoll.go:351
#	0x102bc6ca7	internal/poll.(*pollDesc).wait+0x27						/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/internal/poll/fd_poll_runtime.go:84
#	0x102bc7f5b	internal/poll.(*pollDesc).waitRead+0x1fb					/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/internal/poll/fd_poll_runtime.go:89
#	0x102bc7f4c	internal/poll.(*FD).Read+0x1ec							/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/internal/poll/fd_unix.go:165
#	0x102d11377	net.(*netFD).Read+0x27								/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/fd_posix.go:55
#	0x102d1f8c3	net.(*conn).Read+0x33								/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/net.go:194
#	0x103443277	github.com/soheilhy/cmux.(*bufferedReader).Read+0x137				/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/github.com/soheilhy/[email protected]/buffer.go:53
#	0x1034445e7	github.com/soheilhy/cmux.(*MuxConn).Read+0x27					/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/github.com/soheilhy/[email protected]/cmux.go:297
#	0x102db466f	bufio.(*Reader).Read+0x1af							/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/bufio/bufio.go:245
#	0x102bbe6df	io.ReadAtLeast+0x9f								/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/io/io.go:335
#	0x102fa9b07	io.ReadFull+0x57								/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/io/io.go:354
#	0x102fa9adc	golang.org/x/net/http2.readFrameHeader+0x2c					/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/x/[email protected]/http2/frame.go:237
#	0x102faa1d3	golang.org/x/net/http2.(*Framer).ReadFrame+0x73					/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/x/[email protected]/http2/frame.go:498
#	0x102ff82b7	google.golang.org/grpc/internal/transport.(*http2Server).HandleStreams+0x107	/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/google.golang.org/[email protected]/internal/transport/http2_server.go:636
#	0x10306648b	google.golang.org/grpc.(*Server).serveStreams+0x17b				/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/google.golang.org/[email protected]/server.go:979
#	0x103065d83	google.golang.org/grpc.(*Server).handleRawConn.func1+0x43			/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/google.golang.org/[email protected]/server.go:920

800 @ 0x102b82b18 0x102b60cf4 0x102fe1170 0x102fe19e0 0x102ff5848 0x102b8b2b4
#	0x102fe116f	google.golang.org/grpc/internal/transport.(*controlBuffer).get+0x14f	/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/google.golang.org/[email protected]/internal/transport/controlbuf.go:418
#	0x102fe19df	google.golang.org/grpc/internal/transport.(*loopyWriter).run+0x6f	/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/google.golang.org/[email protected]/internal/transport/controlbuf.go:552
#	0x102ff5847	google.golang.org/grpc/internal/transport.NewServerTransport.func2+0xd7	/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/google.golang.org/[email protected]/internal/transport/http2_server.go:336

800 @ 0x102b82b18 0x102b60cf4 0x102ffbcd4 0x102b8b2b4
#	0x102ffbcd3	google.golang.org/grpc/internal/transport.(*http2Server).keepalive+0x163	/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/google.golang.org/[email protected]/internal/transport/http2_server.go:1152

800 @ 0x102b82b18 0x102b60cf4 0x103000b64 0x103000a70 0x103001d70 0x102bbe6e0 0x103001cf4 0x103001cd5 0x103061b04 0x103062330 0x10306299c 0x103076654 0x10341abb4 0x103133d58 0x103424b04 0x103424a90 0x102b8b2b4
#	0x103000b63	google.golang.org/grpc/internal/transport.(*recvBufferReader).read+0x73			/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/google.golang.org/[email protected]/internal/transport/transport.go:179
#	0x103000a6f	google.golang.org/grpc/internal/transport.(*recvBufferReader).Read+0x16f		/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/google.golang.org/[email protected]/internal/transport/transport.go:173
#	0x103001d6f	google.golang.org/grpc/internal/transport.(*transportReader).Read+0x2f			/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/google.golang.org/[email protected]/internal/transport/transport.go:506
#	0x102bbe6df	io.ReadAtLeast+0x9f									/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/io/io.go:335
#	0x103001cf3	io.ReadFull+0x93									/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/io/io.go:354
#	0x103001cd4	google.golang.org/grpc/internal/transport.(*Stream).Read+0x74				/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/google.golang.org/[email protected]/internal/transport/transport.go:490
#	0x103061b03	google.golang.org/grpc.(*parser).recvMsg+0x43						/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/google.golang.org/[email protected]/rpc_util.go:600
#	0x10306232f	google.golang.org/grpc.recvAndDecompress+0x3f						/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/google.golang.org/[email protected]/rpc_util.go:730
#	0x10306299b	google.golang.org/grpc.recv+0x7b							/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/google.golang.org/[email protected]/rpc_util.go:796
#	0x103076653	google.golang.org/grpc.(*serverStream).RecvMsg+0xe3					/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/google.golang.org/[email protected]/stream.go:1716
#	0x10341abb3	github.com/grpc-ecosystem/go-grpc-prometheus.(*monitoredServerStream).RecvMsg+0x33	/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/github.com/grpc-ecosystem/[email protected]/server_metrics.go:164
#	0x103133d57	go.etcd.io/etcd/api/v3/etcdserverpb.(*leaseLeaseKeepAliveServer).Recv+0x57		/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/api/etcdserverpb/rpc.pb.go:6974
#	0x103424b03	go.etcd.io/etcd/server/v3/etcdserver/api/v3rpc.(*LeaseServer).leaseKeepAlive+0x43	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/api/v3rpc/lease.go:112
#	0x103424a8f	go.etcd.io/etcd/server/v3/etcdserver/api/v3rpc.(*LeaseServer).LeaseKeepAlive.func1+0x2f	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/api/v3rpc/lease.go:96

800 @ 0x102b82b18 0x102b60cf4 0x1034249ac 0x103133c5c 0x103417a78 0x10342072c 0x103422a2c 0x10342072c 0x1034205f4 0x10306ae5c 0x10306c508 0x1030666e0 0x102b8b2b4
#	0x1034249ab	go.etcd.io/etcd/server/v3/etcdserver/api/v3rpc.(*LeaseServer).LeaseKeepAlive+0xfb			/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/api/v3rpc/lease.go:98
#	0x103133c5b	go.etcd.io/etcd/api/v3/etcdserverpb._Lease_LeaseKeepAlive_Handler+0xdb					/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/api/etcdserverpb/rpc.pb.go:6955
#	0x103417a77	github.com/grpc-ecosystem/go-grpc-prometheus.init.(*ServerMetrics).StreamServerInterceptor.func4+0xc7	/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/github.com/grpc-ecosystem/[email protected]/server_metrics.go:121
#	0x10342072b	go.etcd.io/etcd/server/v3/etcdserver/api/v3rpc.Server.ChainStreamServer.func7.1.1+0x3b			/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/github.com/grpc-ecosystem/[email protected]/chain.go:49
#	0x103422a2b	go.etcd.io/etcd/server/v3/etcdserver/api/v3rpc.newStreamInterceptor.func1+0x44b				/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/api/v3rpc/interceptor.go:252
#	0x10342072b	go.etcd.io/etcd/server/v3/etcdserver/api/v3rpc.Server.ChainStreamServer.func7.1.1+0x3b			/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/github.com/grpc-ecosystem/[email protected]/chain.go:49
#	0x1034205f3	go.etcd.io/etcd/server/v3/etcdserver/api/v3rpc.Server.ChainStreamServer.func7+0xb3			/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/github.com/grpc-ecosystem/[email protected]/chain.go:58
#	0x10306ae5b	google.golang.org/grpc.(*Server).processStreamingRPC+0xf1b						/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/google.golang.org/[email protected]/server.go:1644
#	0x10306c507	google.golang.org/grpc.(*Server).handleStream+0x917							/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/google.golang.org/[email protected]/server.go:1741
#	0x1030666df	google.golang.org/grpc.(*Server).serveStreams.func1.1+0x7f						/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/google.golang.org/[email protected]/server.go:986

456 @ 0x102b82b18 0x102b1abc8 0x102b1a7f8 0x103240efc 0x103240dd8 0x102b8b2b4
#	0x103240efb	go.etcd.io/etcd/client/v3.(*watchGrpcStream).serveSubstream.func1+0x6b	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/client/v3/watch.go:797
#	0x103240dd7	go.etcd.io/etcd/client/v3.(*watchGrpcStream).serveSubstream+0x3f7	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/client/v3/watch.go:870

456 @ 0x102b82b18 0x102b1bafc 0x102b1b6b4 0x10340b660 0x10340ba18 0x103409f88 0x10346e8d8 0x10342fd40 0x103417b3c 0x1034209cc 0x103420bd0 0x1034209cc 0x103420cf0 0x1034209cc 0x103420894 0x10342fb98 0x10306846c 0x10306c53c 0x1030666e0 0x102b8b2b4
#	0x10340b65f	go.etcd.io/etcd/client/v3/concurrency.waitDelete+0x12f							/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/client/v3/concurrency/key.go:32
#	0x10340ba17	go.etcd.io/etcd/client/v3/concurrency.waitDeletes+0x207							/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/client/v3/concurrency/key.go:61
#	0x103409f87	go.etcd.io/etcd/client/v3/concurrency.(*Election).Campaign+0x507					/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/client/v3/concurrency/election.go:93
#	0x10346e8d7	go.etcd.io/etcd/server/v3/etcdserver/api/v3election.(*electionServer).Campaign+0xb7			/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/api/v3election/election.go:44
#	0x10342fd3f	go.etcd.io/etcd/server/v3/etcdserver/api/v3election/v3electionpb._Election_Campaign_Handler.func1+0xcf	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/api/v3election/v3electionpb/v3election.pb.go:753
#	0x103417b3b	github.com/grpc-ecosystem/go-grpc-prometheus.init.(*ServerMetrics).UnaryServerInterceptor.func3+0x6b	/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/github.com/grpc-ecosystem/[email protected]/server_metrics.go:107
#	0x1034209cb	go.etcd.io/etcd/server/v3/etcdserver/api/v3rpc.Server.ChainUnaryServer.func5.1.1+0x3b			/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/github.com/grpc-ecosystem/[email protected]/chain.go:25
#	0x103420bcf	go.etcd.io/etcd/server/v3/etcdserver/api/v3rpc.Server.newUnaryInterceptor.func4+0x1cf			/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/api/v3rpc/interceptor.go:71
#	0x1034209cb	go.etcd.io/etcd/server/v3/etcdserver/api/v3rpc.Server.ChainUnaryServer.func5.1.1+0x3b			/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/github.com/grpc-ecosystem/[email protected]/chain.go:25
#	0x103420cef	go.etcd.io/etcd/server/v3/etcdserver/api/v3rpc.Server.newLogUnaryInterceptor.func3+0x7f			/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/api/v3rpc/interceptor.go:78
#	0x1034209cb	go.etcd.io/etcd/server/v3/etcdserver/api/v3rpc.Server.ChainUnaryServer.func5.1.1+0x3b			/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/github.com/grpc-ecosystem/[email protected]/chain.go:25
#	0x103420893	go.etcd.io/etcd/server/v3/etcdserver/api/v3rpc.Server.ChainUnaryServer.func5+0xb3			/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/github.com/grpc-ecosystem/[email protected]/chain.go:34
#	0x10342fb97	go.etcd.io/etcd/server/v3/etcdserver/api/v3election/v3electionpb._Election_Campaign_Handler+0x147	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/api/v3election/v3electionpb/v3election.pb.go:755
#	0x10306846b	google.golang.org/grpc.(*Server).processUnaryRPC+0xb2b							/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/google.golang.org/[email protected]/server.go:1343
#	0x10306c53b	google.golang.org/grpc.(*Server).handleStream+0x94b							/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/google.golang.org/[email protected]/server.go:1737
#	0x1030666df	google.golang.org/grpc.(*Server).serveStreams.func1.1+0x7f						/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/google.golang.org/[email protected]/server.go:986

456 @ 0x102b82b18 0x102b61b64 0x102b61b45 0x102b8427c 0x102b96870 0x103066494 0x103065d84 0x102b8b2b4
#	0x102b8427b	sync.runtime_SemacquireWaitGroup+0x2b				/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/runtime/sema.go:110
#	0x102b9686f	sync.(*WaitGroup).Wait+0x6f					/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/sync/waitgroup.go:118
#	0x103066493	google.golang.org/grpc.(*Server).serveStreams+0x183		/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/google.golang.org/[email protected]/server.go:999
#	0x103065d83	google.golang.org/grpc.(*Server).handleRawConn.func1+0x43	/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/google.golang.org/[email protected]/server.go:920

8 @ 0x102b82b18 0x102b60cf4 0x10327b5a4 0x102b8b2b4
#	0x10327b5a3	go.etcd.io/etcd/server/v3/etcdserver/api/rafthttp.(*pipeline).handle+0xb3	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/api/rafthttp/pipeline.go:97

6 @ 0x102b82b18 0x102b60cf4 0x102e4892c 0x102b8b2b4
#	0x102e4892b	net/http.(*persistConn).writeLoop+0x9b	/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/http/transport.go:2590

4 @ 0x102b82b18 0x102b1bafc 0x102b1b694 0x103275a30 0x102e30184 0x102e30184 0x1034964c4 0x102e2e5f8 0x102e4cf0c 0x102e2cd9c 0x102b8b2b4
#	0x103275a2f	go.etcd.io/etcd/server/v3/etcdserver/api/rafthttp.(*streamHandler).ServeHTTP+0x12df	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/api/rafthttp/http.go:451
#	0x102e30183	net/http.(*ServeMux).ServeHTTP+0x1b3							/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/http/server.go:2822
#	0x102e30183	net/http.(*ServeMux).ServeHTTP+0x1b3							/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/http/server.go:2822
#	0x1034964c3	go.etcd.io/etcd/server/v3/embed.(*Etcd).servePeers.grpcHandlerFunc.func6+0xa3		/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/embed/serve.go:279
#	0x102e2e5f7	net/http.HandlerFunc.ServeHTTP+0x37							/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/http/server.go:2294
#	0x102e4cf0b	net/http.serverHandler.ServeHTTP+0xbb							/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/http/server.go:3301
#	0x102e2cd9b	net/http.(*conn).serve+0x52b								/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/http/server.go:2102

4 @ 0x102b82b18 0x102b60cf4 0x102e47bf4 0x102b8b2b4
#	0x102e47bf3	net/http.(*persistConn).readLoop+0xb03	/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/http/transport.go:2395

4 @ 0x102b82b18 0x102b60cf4 0x10326e850 0x102b8b2b4
#	0x10326e84f	github.com/xiang90/probing.(*prober).AddHTTP.func1+0x8f	/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/github.com/xiang90/[email protected]/prober.go:56

4 @ 0x102b82b18 0x102b60cf4 0x10327c4d8 0x102b8b2b4
#	0x10327c4d7	go.etcd.io/etcd/server/v3/etcdserver/api/rafthttp.monitorProbingStatus+0x127	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/api/rafthttp/probing_status.go:64

4 @ 0x102b82b18 0x102b60cf4 0x10327f80c 0x102b8b2b4
#	0x10327f80b	go.etcd.io/etcd/server/v3/etcdserver/api/rafthttp.(*streamWriter).run+0x2cb	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/api/rafthttp/stream.go:177

3 @ 0x102b82b18 0x102b467a8 0x102b81cd0 0x102bc6ca8 0x102bc7f5c 0x102bc7f4d 0x102d11378 0x102d1f8c4 0x102e93e3c 0x103443278 0x1034445e8 0x102e27710 0x102b8b2b4
#	0x102b81ccf	internal/poll.runtime_pollWait+0x9f				/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/runtime/netpoll.go:351
#	0x102bc6ca7	internal/poll.(*pollDesc).wait+0x27				/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/internal/poll/fd_poll_runtime.go:84
#	0x102bc7f5b	internal/poll.(*pollDesc).waitRead+0x1fb			/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/internal/poll/fd_poll_runtime.go:89
#	0x102bc7f4c	internal/poll.(*FD).Read+0x1ec					/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/internal/poll/fd_unix.go:165
#	0x102d11377	net.(*netFD).Read+0x27						/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/fd_posix.go:55
#	0x102d1f8c3	net.(*conn).Read+0x33						/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/net.go:194
#	0x102e93e3b	go.etcd.io/etcd/client/pkg/v3/transport.timeoutConn.Read+0xab	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/client/pkg/transport/timeout_conn.go:43
#	0x103443277	github.com/soheilhy/cmux.(*bufferedReader).Read+0x137		/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/github.com/soheilhy/[email protected]/buffer.go:53
#	0x1034445e7	github.com/soheilhy/cmux.(*MuxConn).Read+0x27			/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/github.com/soheilhy/[email protected]/cmux.go:297
#	0x102e2770f	net/http.(*connReader).backgroundRead+0x3f			/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/http/server.go:690

3 @ 0x102b82b18 0x102b60cf4 0x102e8a900 0x102b8b2b4
#	0x102e8a8ff	go.etcd.io/etcd/client/pkg/v3/fileutil.purgeFile.func1+0x2df	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/client/pkg/fileutil/purge.go:104

3 @ 0x102b82b18 0x102b844d0 0x102b844b1 0x102b946a4 0x102e27940 0x102e2c0f0 0x102e2cdc4 0x102b8b2b4
#	0x102b844b0	sync.runtime_notifyListWait+0x130		/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/runtime/sema.go:597
#	0x102b946a3	sync.(*Cond).Wait+0xc3				/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/sync/cond.go:71
#	0x102e2793f	net/http.(*connReader).abortPendingRead+0xaf	/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/http/server.go:738
#	0x102e2c0ef	net/http.(*response).finishRequest+0x7f		/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/http/server.go:1720
#	0x102e2cdc3	net/http.(*conn).serve+0x553			/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/http/server.go:2108

2 @ 0x102b82b18 0x102b467a8 0x102b81cd0 0x102bc6ca8 0x102bc7f5c 0x102bc7f4d 0x102d11378 0x102d1f8c4 0x102e4656c 0x102db40a8 0x102db4210 0x102e4721c 0x102b8b2b4
#	0x102b81ccf	internal/poll.runtime_pollWait+0x9f		/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/runtime/netpoll.go:351
#	0x102bc6ca7	internal/poll.(*pollDesc).wait+0x27		/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/internal/poll/fd_poll_runtime.go:84
#	0x102bc7f5b	internal/poll.(*pollDesc).waitRead+0x1fb	/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/internal/poll/fd_poll_runtime.go:89
#	0x102bc7f4c	internal/poll.(*FD).Read+0x1ec			/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/internal/poll/fd_unix.go:165
#	0x102d11377	net.(*netFD).Read+0x27				/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/fd_posix.go:55
#	0x102d1f8c3	net.(*conn).Read+0x33				/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/net.go:194
#	0x102e4656b	net/http.(*persistConn).Read+0x4b		/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/http/transport.go:2122
#	0x102db40a7	bufio.(*Reader).fill+0xf7			/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/bufio/bufio.go:113
#	0x102db420f	bufio.(*Reader).Peek+0x5f			/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/bufio/bufio.go:152
#	0x102e4721b	net/http.(*persistConn).readLoop+0x12b		/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/http/transport.go:2275

2 @ 0x102b82b18 0x102b467a8 0x102b81cd0 0x102bc6ca8 0x102bc7f5c 0x102bc7f4d 0x102d11378 0x102d1f8c4 0x102e4656c 0x102db40a8 0x102db4c90 0x102ddf5c0 0x102ddefb4 0x102ddf338 0x102e3b70c 0x102e3b5bc 0x102e4a2c0 0x102bbe6e0 0x102c5746c 0x102c5746d 0x103276ea0 0x103276d84 0x103282f78 0x1032820c8 0x102b8b2b4
#	0x102b81ccf	internal/poll.runtime_pollWait+0x9f							/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/runtime/netpoll.go:351
#	0x102bc6ca7	internal/poll.(*pollDesc).wait+0x27							/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/internal/poll/fd_poll_runtime.go:84
#	0x102bc7f5b	internal/poll.(*pollDesc).waitRead+0x1fb						/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/internal/poll/fd_poll_runtime.go:89
#	0x102bc7f4c	internal/poll.(*FD).Read+0x1ec								/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/internal/poll/fd_unix.go:165
#	0x102d11377	net.(*netFD).Read+0x27									/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/fd_posix.go:55
#	0x102d1f8c3	net.(*conn).Read+0x33									/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/net.go:194
#	0x102e4656b	net/http.(*persistConn).Read+0x4b							/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/http/transport.go:2122
#	0x102db40a7	bufio.(*Reader).fill+0xf7								/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/bufio/bufio.go:113
#	0x102db4c8f	bufio.(*Reader).ReadSlice+0x2f								/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/bufio/bufio.go:380
#	0x102ddf5bf	net/http/internal.readChunkLine+0x1f							/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/http/internal/chunked.go:156
#	0x102ddefb3	net/http/internal.(*chunkedReader).beginChunk+0x23					/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/http/internal/chunked.go:49
#	0x102ddf337	net/http/internal.(*chunkedReader).Read+0x127						/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/http/internal/chunked.go:125
#	0x102e3b70b	net/http.(*body).readLocked+0x3b							/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/http/transfer.go:845
#	0x102e3b5bb	net/http.(*body).Read+0x11b								/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/http/transfer.go:837
#	0x102e4a2bf	net/http.(*bodyEOFSignal).Read+0x16f							/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/http/transport.go:2984
#	0x102bbe6df	io.ReadAtLeast+0x9f									/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/io/io.go:335
#	0x102c5746b	io.ReadFull+0x7b									/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/io/io.go:354
#	0x102c5746c	encoding/binary.Read+0x7c								/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/encoding/binary/binary.go:262
#	0x103276e9f	go.etcd.io/etcd/server/v3/etcdserver/api/rafthttp.(*messageDecoder).decodeLimit+0x8f	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/api/rafthttp/msg_codec.go:57
#	0x103276d83	go.etcd.io/etcd/server/v3/etcdserver/api/rafthttp.(*messageDecoder).decode+0x63		/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/api/rafthttp/msg_codec.go:51
#	0x103282f77	go.etcd.io/etcd/server/v3/etcdserver/api/rafthttp.(*streamReader).decodeLoop+0x4b7	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/api/rafthttp/stream.go:493
#	0x1032820c7	go.etcd.io/etcd/server/v3/etcdserver/api/rafthttp.(*streamReader).run+0x627		/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/api/rafthttp/stream.go:419

2 @ 0x102b82b18 0x102b467a8 0x102b81cd0 0x102bc6ca8 0x102bc7f5c 0x102bc7f4d 0x102d11378 0x102d1f8c4 0x102e4656c 0x102db40a8 0x102db4c90 0x102ddf5c0 0x102ddefb4 0x102ddf338 0x102e3b70c 0x102e3b5bc 0x102e4a2c0 0x102bbe6e0 0x103277688 0x103277631 0x103282f78 0x1032820c8 0x102b8b2b4
#	0x102b81ccf	internal/poll.runtime_pollWait+0x9f							/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/runtime/netpoll.go:351
#	0x102bc6ca7	internal/poll.(*pollDesc).wait+0x27							/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/internal/poll/fd_poll_runtime.go:84
#	0x102bc7f5b	internal/poll.(*pollDesc).waitRead+0x1fb						/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/internal/poll/fd_poll_runtime.go:89
#	0x102bc7f4c	internal/poll.(*FD).Read+0x1ec								/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/internal/poll/fd_unix.go:165
#	0x102d11377	net.(*netFD).Read+0x27									/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/fd_posix.go:55
#	0x102d1f8c3	net.(*conn).Read+0x33									/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/net.go:194
#	0x102e4656b	net/http.(*persistConn).Read+0x4b							/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/http/transport.go:2122
#	0x102db40a7	bufio.(*Reader).fill+0xf7								/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/bufio/bufio.go:113
#	0x102db4c8f	bufio.(*Reader).ReadSlice+0x2f								/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/bufio/bufio.go:380
#	0x102ddf5bf	net/http/internal.readChunkLine+0x1f							/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/http/internal/chunked.go:156
#	0x102ddefb3	net/http/internal.(*chunkedReader).beginChunk+0x23					/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/http/internal/chunked.go:49
#	0x102ddf337	net/http/internal.(*chunkedReader).Read+0x127						/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/http/internal/chunked.go:125
#	0x102e3b70b	net/http.(*body).readLocked+0x3b							/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/http/transfer.go:845
#	0x102e3b5bb	net/http.(*body).Read+0x11b								/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/http/transfer.go:837
#	0x102e4a2bf	net/http.(*bodyEOFSignal).Read+0x16f							/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/http/transport.go:2984
#	0x102bbe6df	io.ReadAtLeast+0x9f									/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/io/io.go:335
#	0x103277687	io.ReadFull+0x77									/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/io/io.go:354
#	0x103277630	go.etcd.io/etcd/server/v3/etcdserver/api/rafthttp.(*msgAppV2Decoder).decode+0x20	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/api/rafthttp/msgappv2_codec.go:179
#	0x103282f77	go.etcd.io/etcd/server/v3/etcdserver/api/rafthttp.(*streamReader).decodeLoop+0x4b7	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/api/rafthttp/stream.go:493
#	0x1032820c7	go.etcd.io/etcd/server/v3/etcdserver/api/rafthttp.(*streamReader).run+0x627		/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/api/rafthttp/stream.go:419

2 @ 0x102b82b18 0x102b60cf4 0x1032272e8 0x102b8b2b4
#	0x1032272e7	go.etcd.io/etcd/pkg/v3/schedule.(*fifo).run+0x247	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/pkg/schedule/schedule.go:143

2 @ 0x102b82b18 0x102b60cf4 0x103278838 0x102b8b2b4
#	0x103278837	go.etcd.io/etcd/server/v3/etcdserver/api/rafthttp.startPeer.func3+0xb7	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/api/rafthttp/peer.go:194

2 @ 0x102b82b18 0x102b60cf4 0x103278a48 0x102b8b2b4
#	0x103278a47	go.etcd.io/etcd/server/v3/etcdserver/api/rafthttp.startPeer.func2+0xb7	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/api/rafthttp/peer.go:176

2 @ 0x102b82b18 0x102b60cf4 0x103423024 0x1033301ec 0x102b8b2b4
#	0x103423023	go.etcd.io/etcd/server/v3/etcdserver/api/v3rpc.monitorLeader.func1+0xa3	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/api/v3rpc/interceptor.go:316
#	0x1033301eb	go.etcd.io/etcd/server/v3/etcdserver.(*EtcdServer).GoAttach.func1+0x5b	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/server.go:2803

1 @ 0x102b1adac 0x102b1c024 0x10325c0bc 0x10325ac50 0x10325a874 0x102b8b2b4
#	0x10325c0bb	go.etcd.io/etcd/server/v3/mvcc.(*watcher).send+0xab			/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/mvcc/watchable_store.go:573
#	0x10325ac4f	go.etcd.io/etcd/server/v3/mvcc.(*watchableStore).moveVictims+0x32f	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/mvcc/watchable_store.go:283
#	0x10325a873	go.etcd.io/etcd/server/v3/mvcc.(*watchableStore).syncVictimsLoop+0x183	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/mvcc/watchable_store.go:249

1 @ 0x102b42e0c 0x102b81844 0x1033c6d84 0x1033c6bb0 0x1033c429c 0x1033d8034 0x102e30184 0x10349c974 0x102e4cf0c 0x102e2cd9c 0x102b8b2b4
#	0x1033c6d83	runtime/pprof.writeRuntimeProfile+0xb3					/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/runtime/pprof/pprof.go:796
#	0x1033c6baf	runtime/pprof.writeGoroutine+0x4f					/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/runtime/pprof/pprof.go:755
#	0x1033c429b	runtime/pprof.(*Profile).WriteTo+0x14b					/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/runtime/pprof/pprof.go:377
#	0x1033d8033	net/http/pprof.handler.ServeHTTP+0x443					/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/http/pprof/pprof.go:272
#	0x102e30183	net/http.(*ServeMux).ServeHTTP+0x1b3					/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/http/server.go:2822
#	0x10349c973	go.etcd.io/etcd/server/v3/embed.(*accessController).ServeHTTP+0x403	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/embed/serve.go:427
#	0x102e4cf0b	net/http.serverHandler.ServeHTTP+0xbb					/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/http/server.go:3301
#	0x102e2cd9b	net/http.(*conn).serve+0x52b						/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/http/server.go:2102

1 @ 0x102b47840 0x102b84d8c 0x1034a01cc 0x102b8b2b4
#	0x102b84d8b	os/signal.signal_recv+0x2b	/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/runtime/sigqueue.go:149
#	0x1034a01cb	os/signal.loop+0x1b		/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/os/signal/signal_unix.go:23

1 @ 0x102b82b18 0x102b1abc8 0x102b1a7f8 0x1034296f0 0x1034290e4 0x102b8b2b4
#	0x1034296ef	go.etcd.io/etcd/server/v3/etcdserver/api/v3rpc.(*serverWatchStream).recvLoop+0x1ff	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/api/v3rpc/watch.go:348
#	0x1034290e3	go.etcd.io/etcd/server/v3/etcdserver/api/v3rpc.(*watchServer).Watch.func2+0x43		/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/api/v3rpc/watch.go:191

1 @ 0x102b82b18 0x102b1bafc 0x102b1b694 0x10349c048 0x102b8b2b4
#	0x10349c047	go.etcd.io/etcd/server/v3/embed.(*serveCtx).registerGateway.func1+0x47	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/embed/serve.go:311

1 @ 0x102b82b18 0x102b1bafc 0x102b1b694 0x1034a05f8 0x102b8b2b4
#	0x1034a05f7	go.etcd.io/etcd/pkg/v3/osutil.HandleInterrupts.func1+0x37	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/pkg/osutil/interrupt_unix.go:54

1 @ 0x102b82b18 0x102b1bafc 0x102b1b6b4 0x10336b95c 0x102b8b2b4
#	0x10336b95b	gopkg.in/natefinch/lumberjack%2ev2.(*Logger).millRun+0x4b	/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/gopkg.in/natefinch/[email protected]/lumberjack.go:379

1 @ 0x102b82b18 0x102b467a8 0x102b81cd0 0x102bc6ca8 0x102bc7f5c 0x102bc7f4d 0x102d11378 0x102d1f8c4 0x103443278 0x102bbed70 0x102db40a8 0x102db4c90 0x102db4ec4 0x10349d474 0x103443744 0x103443ef0 0x102b8b2b4
#	0x102b81ccf	internal/poll.runtime_pollWait+0x9f						/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/runtime/netpoll.go:351
#	0x102bc6ca7	internal/poll.(*pollDesc).wait+0x27						/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/internal/poll/fd_poll_runtime.go:84
#	0x102bc7f5b	internal/poll.(*pollDesc).waitRead+0x1fb					/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/internal/poll/fd_poll_runtime.go:89
#	0x102bc7f4c	internal/poll.(*FD).Read+0x1ec							/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/internal/poll/fd_unix.go:165
#	0x102d11377	net.(*netFD).Read+0x27								/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/fd_posix.go:55
#	0x102d1f8c3	net.(*conn).Read+0x33								/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/net.go:194
#	0x103443277	github.com/soheilhy/cmux.(*bufferedReader).Read+0x137				/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/github.com/soheilhy/[email protected]/buffer.go:53
#	0x102bbed6f	io.(*LimitedReader).Read+0x4f							/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/io/io.go:479
#	0x102db40a7	bufio.(*Reader).fill+0xf7							/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/bufio/bufio.go:113
#	0x102db4c8f	bufio.(*Reader).ReadSlice+0x2f							/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/bufio/bufio.go:380
#	0x102db4ec3	bufio.(*Reader).ReadLine+0x23							/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/bufio/bufio.go:409
#	0x10349d473	go.etcd.io/etcd/server/v3/embed.(*serveCtx).serve.HTTP1.func9+0x73		/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/github.com/soheilhy/[email protected]/matchers.go:94
#	0x103443743	github.com/soheilhy/cmux.(*cMux).Match.matchersToMatchWriters.func1+0x33	/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/github.com/soheilhy/[email protected]/cmux.go:128
#	0x103443eef	github.com/soheilhy/cmux.(*cMux).serve+0x2af					/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/github.com/soheilhy/[email protected]/cmux.go:192

1 @ 0x102b82b18 0x102b467a8 0x102b81cd0 0x102bc6ca8 0x102bcb57c 0x102bcb56d 0x102d12da8 0x102d28784 0x102d27a6c 0x102e8e468 0x102e93fac 0x1034439cc 0x103496378 0x103495f44 0x102b8b2b4
#	0x102b81ccf	internal/poll.runtime_pollWait+0x9f						/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/runtime/netpoll.go:351
#	0x102bc6ca7	internal/poll.(*pollDesc).wait+0x27						/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/internal/poll/fd_poll_runtime.go:84
#	0x102bcb57b	internal/poll.(*pollDesc).waitRead+0x24b					/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/internal/poll/fd_poll_runtime.go:89
#	0x102bcb56c	internal/poll.(*FD).Accept+0x23c						/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/internal/poll/fd_unix.go:620
#	0x102d12da7	net.(*netFD).accept+0x27							/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/fd_unix.go:172
#	0x102d28783	net.(*TCPListener).accept+0x23							/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/tcpsock_posix.go:159
#	0x102d27a6b	net.(*TCPListener).Accept+0x2b							/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/tcpsock.go:380
#	0x102e8e467	go.etcd.io/etcd/client/pkg/v3/transport.(*keepaliveListener).Accept+0x27	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/client/pkg/transport/keepalive_listener.go:52
#	0x102e93fab	go.etcd.io/etcd/client/pkg/v3/transport.(*rwTimeoutListener).Accept+0x2b	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/client/pkg/transport/timeout_listener.go:36
#	0x1034439cb	github.com/soheilhy/cmux.(*cMux).Serve+0x7b					/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/github.com/soheilhy/[email protected]/cmux.go:170
#	0x103496377	go.etcd.io/etcd/server/v3/embed.(*Etcd).servePeers.func1+0x107			/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/embed/etcd.go:573
#	0x103495f43	go.etcd.io/etcd/server/v3/embed.(*Etcd).servePeers.func3+0x103			/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/embed/etcd.go:601

1 @ 0x102b82b18 0x102b467a8 0x102b81cd0 0x102bc6ca8 0x102bcb57c 0x102bcb56d 0x102d12da8 0x102d28784 0x102d27a6c 0x102e8e468 0x1034439cc 0x10349aa38 0x103498854 0x102b8b2b4
#	0x102b81ccf	internal/poll.runtime_pollWait+0x9f						/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/runtime/netpoll.go:351
#	0x102bc6ca7	internal/poll.(*pollDesc).wait+0x27						/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/internal/poll/fd_poll_runtime.go:84
#	0x102bcb57b	internal/poll.(*pollDesc).waitRead+0x24b					/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/internal/poll/fd_poll_runtime.go:89
#	0x102bcb56c	internal/poll.(*FD).Accept+0x23c						/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/internal/poll/fd_unix.go:620
#	0x102d12da7	net.(*netFD).accept+0x27							/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/fd_unix.go:172
#	0x102d28783	net.(*TCPListener).accept+0x23							/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/tcpsock_posix.go:159
#	0x102d27a6b	net.(*TCPListener).Accept+0x2b							/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/tcpsock.go:380
#	0x102e8e467	go.etcd.io/etcd/client/pkg/v3/transport.(*keepaliveListener).Accept+0x27	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/client/pkg/transport/keepalive_listener.go:52
#	0x1034439cb	github.com/soheilhy/cmux.(*cMux).Serve+0x7b					/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/github.com/soheilhy/[email protected]/cmux.go:170
#	0x10349aa37	go.etcd.io/etcd/server/v3/embed.(*serveCtx).serve+0xc47				/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/embed/serve.go:257
#	0x103498853	go.etcd.io/etcd/server/v3/embed.(*Etcd).serveClients.func1+0xc3			/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/embed/etcd.go:780

1 @ 0x102b82b18 0x102b60cf4 0x102c8d598 0x102b8b2b4
#	0x102c8d597	context.(*cancelCtx).propagateCancel.func2+0x87	/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/context/context.go:514

1 @ 0x102b82b18 0x102b60cf4 0x102f7b814 0x102b8b2b4
#	0x102f7b813	google.golang.org/grpc/internal/grpcsync.(*CallbackSerializer).run+0xc3	/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/google.golang.org/[email protected]/internal/grpcsync/callback_serializer.go:83

1 @ 0x102b82b18 0x102b60cf4 0x1031b540c 0x102b8b2b4
#	0x1031b540b	go.etcd.io/etcd/server/v3/mvcc/backend.(*backend).run+0xcb	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/mvcc/backend/backend.go:421

1 @ 0x102b82b18 0x102b60cf4 0x1031eeb2c 0x102b8b2b4
#	0x1031eeb2b	go.etcd.io/etcd/raft/v3.(*node).run+0x57b	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/raft/node.go:341

1 @ 0x102b82b18 0x102b60cf4 0x103220674 0x102b8b2b4
#	0x103220673	go.etcd.io/etcd/server/v3/lease.(*lessor).runLoop+0xa3	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/lease/lessor.go:616

1 @ 0x102b82b18 0x102b60cf4 0x103234e78 0x102b8b2b4
#	0x103234e77	go.etcd.io/etcd/client/v3.(*lessor).deadlineLoop+0x67	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/client/v3/lease.go:550

1 @ 0x102b82b18 0x102b60cf4 0x103235228 0x102b8b2b4
#	0x103235227	go.etcd.io/etcd/client/v3.(*lessor).sendKeepAliveLoop+0x1d7	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/client/v3/lease.go:590

1 @ 0x102b82b18 0x102b60cf4 0x103240960 0x102b8b2b4
#	0x10324095f	go.etcd.io/etcd/client/v3.(*watchGrpcStream).serveWatchClient+0x9f	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/client/v3/watch.go:774

1 @ 0x102b82b18 0x102b60cf4 0x10325a640 0x102b8b2b4
#	0x10325a63f	go.etcd.io/etcd/server/v3/mvcc.(*watchableStore).syncWatchersLoop+0x1bf	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/mvcc/watchable_store.go:235

1 @ 0x102b82b18 0x102b60cf4 0x10328d91c 0x102b8b2b4
#	0x10328d91b	go.etcd.io/etcd/server/v3/wal.(*filePipeline).run+0xbb	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/wal/file_pipeline.go:97

1 @ 0x102b82b18 0x102b60cf4 0x1032fe348 0x102b8b2b4
#	0x1032fe347	go.etcd.io/etcd/server/v3/etcdserver/api/v3compactor.(*Periodic).Run.func1+0x1c7	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/api/v3compactor/periodic.go:112

1 @ 0x102b82b18 0x102b60cf4 0x103318fec 0x102b8b2b4
#	0x103318feb	go.etcd.io/etcd/server/v3/etcdserver.(*raftNode).start.func1+0xfb	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/raft.go:174

1 @ 0x102b82b18 0x102b60cf4 0x103323110 0x1033301ec 0x102b8b2b4
#	0x10332310f	go.etcd.io/etcd/server/v3/etcdserver.(*EtcdServer).purgeFile+0x1df	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/server.go:892
#	0x1033301eb	go.etcd.io/etcd/server/v3/etcdserver.(*EtcdServer).GoAttach.func1+0x5b	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/server.go:2803

1 @ 0x102b82b18 0x102b60cf4 0x103324584 0x102b8b2b4
#	0x103324583	go.etcd.io/etcd/server/v3/etcdserver.(*EtcdServer).run+0x633	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/server.go:1133

1 @ 0x102b82b18 0x102b60cf4 0x10332e8a8 0x1033301ec 0x102b8b2b4
#	0x10332e8a7	go.etcd.io/etcd/server/v3/etcdserver.(*EtcdServer).monitorVersions+0x77	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/server.go:2533
#	0x1033301eb	go.etcd.io/etcd/server/v3/etcdserver.(*EtcdServer).GoAttach.func1+0x5b	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/server.go:2803

1 @ 0x102b82b18 0x102b60cf4 0x10332f668 0x1033301ec 0x102b8b2b4
#	0x10332f667	go.etcd.io/etcd/server/v3/etcdserver.(*EtcdServer).monitorDowngrade+0x87	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/server.go:2699
#	0x1033301eb	go.etcd.io/etcd/server/v3/etcdserver.(*EtcdServer).GoAttach.func1+0x5b		/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/server.go:2803

1 @ 0x102b82b18 0x102b60cf4 0x103339a60 0x1033301ec 0x102b8b2b4
#	0x103339a5f	go.etcd.io/etcd/server/v3/etcdserver.(*EtcdServer).linearizableReadLoop+0xbf	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/v3_server.go:806
#	0x1033301eb	go.etcd.io/etcd/server/v3/etcdserver.(*EtcdServer).GoAttach.func1+0x5b		/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/server.go:2803

1 @ 0x102b82b18 0x102b60cf4 0x1034249ac 0x10343e954 0x10343e190 0x102b8b2b4
#	0x1034249ab	go.etcd.io/etcd/server/v3/etcdserver/api/v3rpc.(*LeaseServer).LeaseKeepAlive+0xfb	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/api/v3rpc/lease.go:98
#	0x10343e953	go.etcd.io/etcd/server/v3/proxy/grpcproxy/adapter.(*ls2lc).LeaseKeepAlive.func1+0xb3	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/proxy/grpcproxy/adapter/lease_client_adapter.go:43
#	0x10343e18f	go.etcd.io/etcd/server/v3/proxy/grpcproxy/adapter.newPipeStream.func1+0x7f		/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/proxy/grpcproxy/adapter/chan_stream.go:156

1 @ 0x102b82b18 0x102b60cf4 0x103428f84 0x10343f5b4 0x10343e190 0x102b8b2b4
#	0x103428f83	go.etcd.io/etcd/server/v3/etcdserver/api/v3rpc.(*watchServer).Watch+0x2a3	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/api/v3rpc/watch.go:211
#	0x10343f5b3	go.etcd.io/etcd/server/v3/proxy/grpcproxy/adapter.(*ws2wc).Watch.func1+0xb3	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/proxy/grpcproxy/adapter/watch_client_adapter.go:35
#	0x10343e18f	go.etcd.io/etcd/server/v3/proxy/grpcproxy/adapter.newPipeStream.func1+0x7f	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/proxy/grpcproxy/adapter/chan_stream.go:156

1 @ 0x102b82b18 0x102b60cf4 0x10343daac 0x10343f61c 0x10323fabc 0x102b8b2b4
#	0x10343daab	go.etcd.io/etcd/server/v3/proxy/grpcproxy/adapter.(*chanStream).SendMsg+0x7b		/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/proxy/grpcproxy/adapter/chan_stream.go:108
#	0x10343f61b	go.etcd.io/etcd/server/v3/proxy/grpcproxy/adapter.(*ws2wcClientStream).Send+0x2b	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/proxy/grpcproxy/adapter/watch_client_adapter.go:47
#	0x10323fabb	go.etcd.io/etcd/client/v3.(*watchGrpcStream).run+0x105b					/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/client/v3/watch.go:692

1 @ 0x102b82b18 0x102b60cf4 0x10343daac 0x10343f714 0x10342a2e0 0x1034293b4 0x102b8b2b4
#	0x10343daab	go.etcd.io/etcd/server/v3/proxy/grpcproxy/adapter.(*chanStream).SendMsg+0x7b		/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/proxy/grpcproxy/adapter/chan_stream.go:108
#	0x10343f713	go.etcd.io/etcd/server/v3/proxy/grpcproxy/adapter.(*ws2wcServerStream).Send+0x33	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/proxy/grpcproxy/adapter/watch_client_adapter.go:58
#	0x10342a2df	go.etcd.io/etcd/server/v3/etcdserver/api/v3rpc.(*serverWatchStream).sendLoop+0x30f	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/api/v3rpc/watch.go:478
#	0x1034293b3	go.etcd.io/etcd/server/v3/etcdserver/api/v3rpc.(*watchServer).Watch.func1+0x23		/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/api/v3rpc/watch.go:181

1 @ 0x102b82b18 0x102b60cf4 0x10343dc0c 0x10343eaf0 0x103234154 0x102b8b2b4
#	0x10343dc0b	go.etcd.io/etcd/server/v3/proxy/grpcproxy/adapter.(*chanStream).RecvMsg+0x9b		/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/proxy/grpcproxy/adapter/chan_stream.go:122
#	0x10343eaef	go.etcd.io/etcd/server/v3/proxy/grpcproxy/adapter.(*ls2lcClientStream).Recv+0x2f	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/proxy/grpcproxy/adapter/lease_client_adapter.go:67
#	0x103234153	go.etcd.io/etcd/client/v3.(*lessor).recvKeepAliveLoop+0x203				/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/client/v3/lease.go:458

1 @ 0x102b82b18 0x102b60cf4 0x10343dc0c 0x10343ec18 0x103424b04 0x103424a90 0x102b8b2b4
#	0x10343dc0b	go.etcd.io/etcd/server/v3/proxy/grpcproxy/adapter.(*chanStream).RecvMsg+0x9b		/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/proxy/grpcproxy/adapter/chan_stream.go:122
#	0x10343ec17	go.etcd.io/etcd/server/v3/proxy/grpcproxy/adapter.(*ls2lcServerStream).Recv+0x57	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/proxy/grpcproxy/adapter/lease_client_adapter.go:78
#	0x103424b03	go.etcd.io/etcd/server/v3/etcdserver/api/v3rpc.(*LeaseServer).leaseKeepAlive+0x43	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/api/v3rpc/lease.go:112
#	0x103424a8f	go.etcd.io/etcd/server/v3/etcdserver/api/v3rpc.(*LeaseServer).LeaseKeepAlive.func1+0x2f	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdserver/api/v3rpc/lease.go:96

1 @ 0x102b82b18 0x102b60cf4 0x103444c34 0x102e31be0 0x102b8b2b4
#	0x103444c33	github.com/soheilhy/cmux.muxListener.Accept+0x63	/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/github.com/soheilhy/[email protected]/cmux.go:262
#	0x102e31bdf	net/http.(*Server).Serve+0x28f				/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/http/server.go:3424

1 @ 0x102b82b18 0x102b60cf4 0x103444c34 0x102e31be0 0x10349baac 0x102b8b2b4
#	0x103444c33	github.com/soheilhy/cmux.muxListener.Accept+0x63		/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/github.com/soheilhy/[email protected]/cmux.go:262
#	0x102e31bdf	net/http.(*Server).Serve+0x28f					/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/net/http/server.go:3424
#	0x10349baab	go.etcd.io/etcd/server/v3/embed.(*serveCtx).serve.func3+0x2b	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/embed/serve.go:175

1 @ 0x102b82b18 0x102b60cf4 0x103444c34 0x103065398 0x102b8b2b4
#	0x103444c33	github.com/soheilhy/cmux.muxListener.Accept+0x63	/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/github.com/soheilhy/[email protected]/cmux.go:262
#	0x103065397	google.golang.org/grpc.(*Server).Serve+0x427		/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/google.golang.org/[email protected]/server.go:852

1 @ 0x102b82b18 0x102b60cf4 0x103444c34 0x103065398 0x10349b9dc 0x102b8b2b4
#	0x103444c33	github.com/soheilhy/cmux.muxListener.Accept+0x63		/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/github.com/soheilhy/[email protected]/cmux.go:262
#	0x103065397	google.golang.org/grpc.(*Server).Serve+0x427			/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/google.golang.org/[email protected]/server.go:852
#	0x10349b9db	go.etcd.io/etcd/server/v3/embed.(*serveCtx).serve.func4+0x2b	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/embed/serve.go:181

1 @ 0x102b82b18 0x102b60cf4 0x1034dabfc 0x1034e2880 0x1034e3844 0x102b4d294 0x102b8b2b4
#	0x1034dabfb	go.etcd.io/etcd/server/v3/etcdmain.startEtcdOrProxyV2+0x1dcb	/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdmain/etcd.go:216
#	0x1034e287f	go.etcd.io/etcd/server/v3/etcdmain.Main+0x11f			/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/etcdmain/main.go:40
#	0x1034e3843	main.main+0x33							/Users/sh01108ml/murray/Go_WorkSpace/src/etcd_community/etcd/server/main.go:31
#	0x102b4d293	runtime.main+0x283						/Users/sh01108ml/murray/Go_WorkSpace/pkg/mod/golang.org/[email protected]/src/runtime/proc.go:283

1 @ 0x102b8b2b4

Relevant log output

Metadata

Metadata

Assignees

No one assigned

    Labels

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions