diff --git a/Dockerfile-test b/Dockerfile-test
index 8e0be6a3f..5eeccb14e 100644
--- a/Dockerfile-test
+++ b/Dockerfile-test
@@ -51,6 +51,7 @@ RUN go get -v -u -tags spell github.com/chzchzchz/goword \
   && go get -v -u honnef.co/go/tools/cmd/staticcheck \
   && go get -v -u github.com/wadey/gocovmerge \
   && go get -v -u github.com/gordonklaus/ineffassign \
+  && go get -v -u github.com/alexkohler/nakedret \
   && /tmp/install-marker.sh amd64 \
   && rm -f /tmp/install-marker.sh \
   && curl -s https://codecov.io/bash >/codecov \
diff --git a/client/keys.go b/client/keys.go
index e8373b945..8b9fd3f87 100644
--- a/client/keys.go
+++ b/client/keys.go
@@ -653,8 +653,7 @@ func unmarshalHTTPResponse(code int, header http.Header, body []byte) (res *Resp
 	default:
 		err = unmarshalFailedKeysResponse(body)
 	}
-
-	return
+	return res, err
 }
 
 func unmarshalSuccessfulKeysResponse(header http.Header, body []byte) (*Response, error) {
diff --git a/clientv3/client.go b/clientv3/client.go
index 3bbe2739a..bc695748a 100644
--- a/clientv3/client.go
+++ b/clientv3/client.go
@@ -192,7 +192,7 @@ func parseEndpoint(endpoint string) (proto string, host string, scheme string) {
 	host = endpoint
 	url, uerr := url.Parse(endpoint)
 	if uerr != nil || !strings.Contains(endpoint, "://") {
-		return
+		return proto, host, scheme
 	}
 	scheme = url.Scheme
 
@@ -206,7 +206,7 @@ func parseEndpoint(endpoint string) (proto string, host string, scheme string) {
 	default:
 		proto, host = "", ""
 	}
-	return
+	return proto, host, scheme
 }
 
 func (c *Client) processCreds(scheme string) (creds *credentials.TransportCredentials) {
@@ -225,7 +225,7 @@ func (c *Client) processCreds(scheme string) (creds *credentials.TransportCreden
 	default:
 		creds = nil
 	}
-	return
+	return creds
 }
 
 // dialSetupOpts gives the dial opts prior to any authentication
diff --git a/contrib/raftexample/raft.go b/contrib/raftexample/raft.go
index b50c5d008..2b7f3207a 100644
--- a/contrib/raftexample/raft.go
+++ b/contrib/raftexample/raft.go
@@ -133,7 +133,7 @@ func (rc *raftNode) entriesToApply(ents []raftpb.Entry) (nents []raftpb.Entry) {
 	if rc.appliedIndex-firstIdx+1 < uint64(len(ents)) {
 		nents = ents[rc.appliedIndex-firstIdx+1:]
 	}
-	return
+	return nents
 }
 
 // publishEntries writes committed log entries to commit channel and returns
diff --git a/embed/etcd.go b/embed/etcd.go
index eb51b23bb..5e00ce13f 100644
--- a/embed/etcd.go
+++ b/embed/etcd.go
@@ -108,10 +108,10 @@ func StartEtcd(inCfg *Config) (e *Etcd, err error) {
 	}()
 
 	if e.Peers, err = startPeerListeners(cfg); err != nil {
-		return
+		return e, err
 	}
 	if e.sctxs, err = startClientListeners(cfg); err != nil {
-		return
+		return e, err
 	}
 	for _, sctx := range e.sctxs {
 		e.Clients = append(e.Clients, sctx.l)
@@ -177,7 +177,7 @@ func StartEtcd(inCfg *Config) (e *Etcd, err error) {
 	}
 
 	if e.Server, err = etcdserver.NewServer(srvcfg); err != nil {
-		return
+		return e, err
 	}
 
 	// buffer channel so goroutines on closed connections won't wait forever
@@ -190,7 +190,7 @@ func StartEtcd(inCfg *Config) (e *Etcd, err error) {
 	var peerTLScfg *tls.Config
 	if !cfg.PeerTLSInfo.Empty() {
 		if peerTLScfg, err = cfg.PeerTLSInfo.ServerConfig(); err != nil {
-			return
+			return e, err
 		}
 	}
 	for _, p := range e.Peers {
@@ -214,10 +214,10 @@ func StartEtcd(inCfg *Config) (e *Etcd, err error) {
 	}
 
 	if err = e.serve(); err != nil {
-		return
+		return e, err
 	}
 	serving = true
-	return
+	return e, nil
 }
 
 // Config returns the current configuration.
diff --git a/etcdctl/ctlv3/command/printer.go b/etcdctl/ctlv3/command/printer.go
index 64c0fb72e..b0c2d6cad 100644
--- a/etcdctl/ctlv3/command/printer.go
+++ b/etcdctl/ctlv3/command/printer.go
@@ -169,7 +169,7 @@ func makeMemberListTable(r v3.MemberListResponse) (hdr []string, rows [][]string
 			strings.Join(m.ClientURLs, ","),
 		})
 	}
-	return
+	return hdr, rows
 }
 
 func makeEndpointStatusTable(statusList []epStatus) (hdr []string, rows [][]string) {
@@ -185,7 +185,7 @@ func makeEndpointStatusTable(statusList []epStatus) (hdr []string, rows [][]stri
 			fmt.Sprint(status.Resp.RaftIndex),
 		})
 	}
-	return
+	return hdr, rows
 }
 
 func makeEndpointHashKVTable(hashList []epHashKV) (hdr []string, rows [][]string) {
@@ -196,7 +196,7 @@ func makeEndpointHashKVTable(hashList []epHashKV) (hdr []string, rows [][]string
 			fmt.Sprint(h.Resp.Hash),
 		})
 	}
-	return
+	return hdr, rows
 }
 
 func makeDBStatusTable(ds dbstatus) (hdr []string, rows [][]string) {
@@ -207,5 +207,5 @@ func makeDBStatusTable(ds dbstatus) (hdr []string, rows [][]string) {
 		fmt.Sprint(ds.TotalKey),
 		humanize.Bytes(uint64(ds.TotalSize)),
 	})
-	return
+	return hdr, rows
 }
diff --git a/etcdserver/raft.go b/etcdserver/raft.go
index 8e9070149..e6bb723cd 100644
--- a/etcdserver/raft.go
+++ b/etcdserver/raft.go
@@ -416,7 +416,7 @@ func startNode(cfg ServerConfig, cl *membership.RaftCluster, ids []types.ID) (id
 	raftStatus = n.Status
 	raftStatusMu.Unlock()
 	advanceTicksForElection(n, c.ElectionTick)
-	return
+	return id, n, s, w
 }
 
 func restartNode(cfg ServerConfig, snapshot *raftpb.Snapshot) (types.ID, *membership.RaftCluster, raft.Node, *raft.MemoryStorage, *wal.WAL) {
diff --git a/etcdserver/storage.go b/etcdserver/storage.go
index aa8f87569..55c2dd4b6 100644
--- a/etcdserver/storage.go
+++ b/etcdserver/storage.go
@@ -94,5 +94,5 @@ func readWAL(waldir string, snap walpb.Snapshot) (w *wal.WAL, id, cid types.ID,
 	pbutil.MustUnmarshal(&metadata, wmetadata)
 	id = types.ID(metadata.NodeID)
 	cid = types.ID(metadata.ClusterID)
-	return
+	return w, id, cid, st, ents
 }
diff --git a/integration/bridge.go b/integration/bridge.go
index 59cebe1f0..9792ba89f 100644
--- a/integration/bridge.go
+++ b/integration/bridge.go
@@ -224,5 +224,5 @@ func (b *bridge) ioCopy(bc *bridgeConn, dst io.Writer, src io.Reader) (err error
 			break
 		}
 	}
-	return
+	return err
 }
diff --git a/integration/embed_test.go b/integration/embed_test.go
index 751494eae..2883ec716 100644
--- a/integration/embed_test.go
+++ b/integration/embed_test.go
@@ -151,7 +151,7 @@ func newEmbedURLs(n int) (urls []url.URL) {
 		u, _ := url.Parse(fmt.Sprintf("unix://localhost:%d%06d", os.Getpid(), i))
 		urls = append(urls, *u)
 	}
-	return
+	return urls
 }
 
 func setupEmbedCfg(cfg *embed.Config, curls []url.URL, purls []url.URL) {
diff --git a/pkg/fileutil/lock_windows.go b/pkg/fileutil/lock_windows.go
index 8698f4a8d..b1817230a 100644
--- a/pkg/fileutil/lock_windows.go
+++ b/pkg/fileutil/lock_windows.go
@@ -121,5 +121,5 @@ func lockFileEx(h syscall.Handle, flags, locklow, lockhigh uint32, ol *syscall.O
 			err = syscall.EINVAL
 		}
 	}
-	return
+	return err
 }
diff --git a/pkg/netutil/routes_linux.go b/pkg/netutil/routes_linux.go
index 5d234d460..797baeb48 100644
--- a/pkg/netutil/routes_linux.go
+++ b/pkg/netutil/routes_linux.go
@@ -246,5 +246,5 @@ func parsePREFSRC(m *syscall.NetlinkMessage) (host string, oif uint32, err error
 	if oif == 0 {
 		err = errNoDefaultRoute
 	}
-	return
+	return host, oif, err
 }
diff --git a/pkg/report/report.go b/pkg/report/report.go
index 77e29838a..36d09b9d5 100644
--- a/pkg/report/report.go
+++ b/pkg/report/report.go
@@ -127,13 +127,13 @@ func copyMap(m map[string]int) (c map[string]int) {
 	for k, v := range m {
 		c[k] = v
 	}
-	return
+	return c
 }
 
 func copyFloats(s []float64) (c []float64) {
 	c = make([]float64, len(s))
 	copy(c, s)
-	return
+	return c
 }
 
 func (r *report) String() (s string) {
@@ -221,7 +221,7 @@ func percentiles(nums []float64) (data []float64) {
 			j++
 		}
 	}
-	return
+	return data
 }
 
 func (r *report) sprintLatencies() string {
diff --git a/pkg/testutil/leak.go b/pkg/testutil/leak.go
index 3bf7e6b67..2ebae1e6e 100644
--- a/pkg/testutil/leak.go
+++ b/pkg/testutil/leak.go
@@ -135,5 +135,5 @@ func interestingGoroutines() (gs []string) {
 		gs = append(gs, stack)
 	}
 	sort.Strings(gs)
-	return
+	return gs
 }
diff --git a/pkg/transport/keepalive_listener.go b/pkg/transport/keepalive_listener.go
index 6ccae4ee4..4ff8e7f00 100644
--- a/pkg/transport/keepalive_listener.go
+++ b/pkg/transport/keepalive_listener.go
@@ -79,7 +79,7 @@ func (l *tlsKeepaliveListener) Accept() (c net.Conn, err error) {
 	kac.SetKeepAlive(true)
 	kac.SetKeepAlivePeriod(30 * time.Second)
 	c = tls.Server(c, l.config)
-	return
+	return c, nil
 }
 
 // NewListener creates a Listener which accepts connections from an inner
diff --git a/pkg/types/set.go b/pkg/types/set.go
index 73ef431be..c111b0c0c 100644
--- a/pkg/types/set.go
+++ b/pkg/types/set.go
@@ -61,7 +61,7 @@ func (us *unsafeSet) Remove(value string) {
 // Contains returns whether the set contains the given value
 func (us *unsafeSet) Contains(value string) (exists bool) {
 	_, exists = us.d[value]
-	return
+	return exists
 }
 
 // ContainsAll returns whether the set contains all given values
@@ -94,7 +94,7 @@ func (us *unsafeSet) Values() (values []string) {
 	for val := range us.d {
 		values = append(values, val)
 	}
-	return
+	return values
 }
 
 // Copy creates a new Set containing the values of the first
diff --git a/store/node.go b/store/node.go
index 541595535..c3c87431c 100644
--- a/store/node.go
+++ b/store/node.go
@@ -350,7 +350,7 @@ func (n *node) Compare(prevValue string, prevIndex uint64) (ok bool, which int)
 	default:
 		which = CompareNotMatch
 	}
-	return
+	return ok, which
 }
 
 // Clone function clone the node recursively and return the new node.
diff --git a/test b/test
index ed190b696..cec89e6d7 100755
--- a/test
+++ b/test
@@ -403,6 +403,17 @@ function fmt_pass {
 		echo "Skipping ineffassign..."
 	fi
 
+	if which nakedret >/dev/null; then
+		echo "Checking nakedret..."
+		nakedretResult=$(nakedret "${STATIC_ANALYSIS_PATHS[@]}" 2>&1 || true)
+		if [ -n "${nakedretResult}" ]; then
+			echo -e "nakedret checking failed:\n${nakedretResult}"
+			exit 255
+		fi
+	else
+		echo "Skipping nakedret..."
+	fi
+
 	echo "Checking for license header..."
 	licRes=""
 	files=$(find . -type f -iname '*.go' ! -path './cmd/*' ! -path './gopath.proto/*')
diff --git a/tools/etcd-dump-db/backend.go b/tools/etcd-dump-db/backend.go
index 618d81149..aff90fc49 100644
--- a/tools/etcd-dump-db/backend.go
+++ b/tools/etcd-dump-db/backend.go
@@ -40,7 +40,7 @@ func getBuckets(dbPath string) (buckets []string, err error) {
 			return nil
 		})
 	})
-	return
+	return buckets, err
 }
 
 func iterateBucket(dbPath, bucket string, limit uint64) (err error) {
@@ -70,7 +70,7 @@ func iterateBucket(dbPath, bucket string, limit uint64) (err error) {
 
 		return nil
 	})
-	return
+	return err
 }
 
 func getHash(dbPath string) (hash uint32, err error) {
diff --git a/tools/etcd-dump-logs/main.go b/tools/etcd-dump-logs/main.go
index 87275c95d..17343180a 100644
--- a/tools/etcd-dump-logs/main.go
+++ b/tools/etcd-dump-logs/main.go
@@ -141,7 +141,7 @@ func parseWALMetadata(b []byte) (id, cid types.ID) {
 	pbutil.MustUnmarshal(&metadata, b)
 	id = types.ID(metadata.NodeID)
 	cid = types.ID(metadata.ClusterID)
-	return
+	return id, cid
 }
 
 func genIDSlice(a []uint64) []types.ID {
diff --git a/wal/decoder.go b/wal/decoder.go
index 97d87d378..6a217f897 100644
--- a/wal/decoder.go
+++ b/wal/decoder.go
@@ -119,7 +119,7 @@ func decodeFrameSize(lenField int64) (recBytes int64, padBytes int64) {
 		// padding is stored in lower 3 bits of length MSB
 		padBytes = int64((uint64(lenField) >> 56) & 0x7)
 	}
-	return
+	return recBytes, padBytes
 }
 
 // isTornEntry determines whether the last entry of the WAL was partially written
diff --git a/wal/encoder.go b/wal/encoder.go
index aac1e197e..e8040b8df 100644
--- a/wal/encoder.go
+++ b/wal/encoder.go
@@ -103,7 +103,7 @@ func encodeFrameSize(dataBytes int) (lenField uint64, padBytes int) {
 	if padBytes != 0 {
 		lenField |= uint64(0x80|padBytes) << 56
 	}
-	return
+	return lenField, padBytes
 }
 
 func (e *encoder) flush() error {
diff --git a/wal/file_pipeline.go b/wal/file_pipeline.go
index 5e32a0693..3a1c57c1c 100644
--- a/wal/file_pipeline.go
+++ b/wal/file_pipeline.go
@@ -55,7 +55,7 @@ func (fp *filePipeline) Open() (f *fileutil.LockedFile, err error) {
 	case f = <-fp.filec:
 	case err = <-fp.errc:
 	}
-	return
+	return f, err
 }
 
 func (fp *filePipeline) Close() error {