From d80a9f515fcba6ff874fde88f7cd1c42f583a59c Mon Sep 17 00:00:00 2001 From: Alex Santamaura Date: Tue, 22 Aug 2023 15:46:08 -0400 Subject: [PATCH] lint: make a smarter mutex lock linter This commit udpates the lock linter to work by keeping track of locks and if we find any if conditions, loops, or function calls before finding a matching unlock it will report. It will also report if the matching unlock is >5 lines away from the lock. It will ignore cases where a `nolint:deferunlock` is present. Release note: None --- build/bazelutil/nogo_config.json | 239 ++++++++++----- pkg/acceptance/localcluster/cluster.go | 8 - pkg/ccl/auditloggingccl/audit_log_config.go | 1 - pkg/ccl/backupccl/backup_test.go | 8 - .../backupccl/backupinfo/manifest_handling.go | 1 - pkg/ccl/backupccl/restore_job.go | 2 - .../backupccl/tenant_backup_nemesis_test.go | 2 - .../changefeedccl/cdctest/schema_registry.go | 1 - .../changefeedccl/changefeed_processors.go | 3 - pkg/ccl/changefeedccl/event_processing.go | 3 - .../changefeedccl/kvevent/blocking_buffer.go | 3 - pkg/ccl/changefeedccl/metrics.go | 1 - pkg/ccl/changefeedccl/schema_registry.go | 1 - .../changefeedccl/schemafeed/schema_feed.go | 3 - pkg/ccl/changefeedccl/sink_kafka.go | 3 - pkg/ccl/changefeedccl/sink_pubsub_v2.go | 2 - pkg/ccl/changefeedccl/sink_test.go | 2 - pkg/ccl/multiregionccl/datadriven_test.go | 2 - pkg/ccl/multiregionccl/region_test.go | 2 - .../tenantcostclient/tenant_side.go | 2 - .../datadriven_test.go | 3 - .../sqlwatcher_test.go | 4 - pkg/ccl/sqlproxyccl/acl/watcher.go | 2 - pkg/ccl/sqlproxyccl/balancer/balancer_test.go | 3 - pkg/ccl/sqlproxyccl/forwarder_test.go | 13 - pkg/ccl/sqlproxyccl/proxy_handler_test.go | 2 - .../tenantdirsvr/test_directory_svr.go | 3 - .../tenantdirsvr/test_static_directory_svr.go | 3 - .../streamingccl/streamclient/client_test.go | 3 - .../streamclient/random_stream_client.go | 1 - .../replication_stream_e2e_test.go | 2 - .../streamproducer/replication_stream_test.go | 2 - pkg/cli/clisqlexec/format_csv_tsv.go | 4 - pkg/cli/clisqlshell/sql.go | 3 - pkg/cli/syncbench/syncbench.go | 2 - pkg/cmd/allocsim/main.go | 2 - pkg/cmd/roachtest/cluster.go | 5 - pkg/cmd/roachtest/test_impl.go | 3 - pkg/cmd/roachtest/test_runner.go | 6 - pkg/cmd/zerosum/main.go | 3 - pkg/config/system.go | 7 - pkg/gossip/client.go | 2 - pkg/gossip/client_test.go | 12 - pkg/gossip/gossip.go | 8 - pkg/gossip/gossip_test.go | 20 -- pkg/gossip/infostore.go | 3 - pkg/gossip/infostore_test.go | 2 - pkg/gossip/server.go | 2 - pkg/internal/rsg/rsg.go | 5 - pkg/internal/sqlsmith/bulkio.go | 5 - pkg/internal/sqlsmith/relational.go | 1 - pkg/internal/sqlsmith/sampler.go | 4 - pkg/internal/sqlsmith/scalar.go | 1 - pkg/internal/sqlsmith/schema.go | 1 - pkg/jobs/adopt.go | 1 - pkg/jobs/job_scheduler.go | 1 - pkg/jobs/jobs_test.go | 9 - pkg/jobs/progress.go | 2 - pkg/jobs/registry.go | 4 - pkg/kv/bulk/buffering_adder.go | 5 - pkg/kv/bulk/sst_batcher.go | 1 - .../kvcoord/dist_sender_mux_rangefeed.go | 3 - .../kvcoord/dist_sender_rangefeed_test.go | 2 - .../kvclient/kvcoord/txn_correctness_test.go | 1 - .../kvcoord/txn_interceptor_heartbeater.go | 1 - .../txn_interceptor_heartbeater_test.go | 9 - .../kvstreamer/requests_provider_test.go | 1 - .../kvstreamer/results_buffer_test.go | 2 - pkg/kv/kvclient/kvstreamer/streamer.go | 3 - pkg/kv/kvclient/kvtenant/connector.go | 2 - pkg/kv/kvclient/rangecache/range_cache.go | 1 - .../kvclient/rangecache/range_cache_test.go | 1 - .../rangefeed/rangefeed_external_test.go | 3 - .../rangefeed/rangefeedcache/watcher.go | 3 - pkg/kv/kvnemesis/applier.go | 1 - pkg/kv/kvnemesis/watcher.go | 2 - .../allocatorimpl/allocator_scorer.go | 3 - .../allocator/allocatorimpl/allocator_test.go | 1 - .../storepool/override_store_pool_test.go | 1 - .../allocator/storepool/store_pool.go | 2 - .../allocator/storepool/store_pool_test.go | 4 - pkg/kv/kvserver/allocator_impl_test.go | 2 - pkg/kv/kvserver/client_merge_test.go | 8 - pkg/kv/kvserver/client_replica_test.go | 5 - .../kvserver/closedts/sidetransport/debug.go | 3 - .../closedts/sidetransport/receiver.go | 3 - .../kvserver/closedts/sidetransport/sender.go | 3 - .../closedts/sidetransport/sender_test.go | 1 - .../kvserver/closedts/tracker/tracker_test.go | 2 - .../concurrency/concurrency_manager_test.go | 6 - pkg/kv/kvserver/concurrency/lock_table.go | 20 -- .../kvserver/concurrency/lock_table_test.go | 1 - .../kvserver/concurrency/lock_table_waiter.go | 2 - pkg/kv/kvserver/helpers_test.go | 1 - pkg/kv/kvserver/idalloc/id_alloc_test.go | 4 - .../intentresolver/intent_resolver.go | 2 - .../intentresolver/intent_resolver_test.go | 6 - .../kvflowhandle/kvflowhandle.go | 2 - pkg/kv/kvserver/liveness/cache.go | 1 - pkg/kv/kvserver/liveness/client_test.go | 2 - pkg/kv/kvserver/liveness/liveness.go | 1 - pkg/kv/kvserver/metrics.go | 2 - pkg/kv/kvserver/mvcc_gc_queue.go | 1 - pkg/kv/kvserver/mvcc_gc_queue_test.go | 2 - pkg/kv/kvserver/protectedts/ptcache/cache.go | 1 - .../ptreconcile/reconciler_test.go | 1 - pkg/kv/kvserver/queue.go | 9 - pkg/kv/kvserver/queue_concurrency_test.go | 1 - pkg/kv/kvserver/queue_test.go | 2 - pkg/kv/kvserver/raft.go | 1 - pkg/kv/kvserver/raft_log_queue.go | 3 - pkg/kv/kvserver/raft_log_queue_test.go | 7 - pkg/kv/kvserver/raft_log_truncator.go | 5 - pkg/kv/kvserver/raft_transport.go | 6 - pkg/kv/kvserver/raftentry/cache.go | 4 - pkg/kv/kvserver/rangefeed/budget.go | 3 - pkg/kv/kvserver/rangefeed/processor_test.go | 1 - pkg/kv/kvserver/rangefeed/registry.go | 4 - pkg/kv/kvserver/replica.go | 3 - pkg/kv/kvserver/replica_app_batch.go | 6 - pkg/kv/kvserver/replica_application_result.go | 4 - .../replica_application_state_machine.go | 4 - .../replica_application_state_machine_test.go | 3 - pkg/kv/kvserver/replica_closedts.go | 1 - .../replica_closedts_internal_test.go | 8 - pkg/kv/kvserver/replica_command.go | 3 - pkg/kv/kvserver/replica_learner_test.go | 6 - pkg/kv/kvserver/replica_metrics.go | 1 - pkg/kv/kvserver/replica_probe_test.go | 1 - pkg/kv/kvserver/replica_proposal_buf_test.go | 1 - pkg/kv/kvserver/replica_proposal_quota.go | 1 - .../replica_protected_timestamp_test.go | 2 - pkg/kv/kvserver/replica_raft.go | 8 - pkg/kv/kvserver/replica_raft_quiesce.go | 2 - pkg/kv/kvserver/replica_raftstorage.go | 6 - pkg/kv/kvserver/replica_range_lease.go | 1 - pkg/kv/kvserver/replica_rangefeed.go | 3 - pkg/kv/kvserver/replica_rankings.go | 2 - pkg/kv/kvserver/replica_send.go | 1 - .../kvserver/replica_sst_snapshot_storage.go | 1 - pkg/kv/kvserver/replica_test.go | 32 -- pkg/kv/kvserver/replica_write.go | 1 - pkg/kv/kvserver/scanner.go | 1 - pkg/kv/kvserver/scanner_test.go | 3 - pkg/kv/kvserver/scheduler.go | 5 - pkg/kv/kvserver/scheduler_test.go | 4 - pkg/kv/kvserver/spanlatch/manager.go | 3 - pkg/kv/kvserver/split_delay_helper.go | 1 - pkg/kv/kvserver/split_queue_test.go | 1 - pkg/kv/kvserver/split_trigger_helper.go | 1 - pkg/kv/kvserver/store.go | 15 - pkg/kv/kvserver/store_create_replica.go | 7 - pkg/kv/kvserver/store_gossip.go | 2 - pkg/kv/kvserver/store_pool_test.go | 1 - pkg/kv/kvserver/store_raft.go | 5 - pkg/kv/kvserver/store_remove_replica.go | 4 - pkg/kv/kvserver/store_snapshot.go | 1 - pkg/kv/kvserver/store_split.go | 1 - pkg/kv/kvserver/store_test.go | 8 - pkg/kv/kvserver/stores_server.go | 1 - pkg/kv/kvserver/ts_maintenance_queue_test.go | 2 - pkg/kv/kvserver/txnwait/queue.go | 10 - pkg/kv/txn.go | 2 - .../authorizer.go | 1 - .../tenantcapabilitieswatcher/watcher.go | 1 - .../tenantcapabilitieswatcher/watcher_test.go | 2 - pkg/obs/event_exporter.go | 1 - pkg/roachprod/install/cluster_synced.go | 6 - pkg/roachprod/roachprod.go | 2 - pkg/roachprod/vm/aws/aws.go | 1 - pkg/roachprod/vm/azure/auth.go | 2 - pkg/roachprod/vm/azure/azure.go | 13 - pkg/rpc/clock_offset_test.go | 4 - pkg/rpc/context.go | 1 - pkg/rpc/context_test.go | 1 - pkg/rpc/nodedialer/nodedialer_test.go | 1 - pkg/rpc/peer.go | 3 - pkg/security/auth.go | 2 - pkg/server/drain.go | 1 - pkg/server/init.go | 2 - pkg/server/node.go | 2 - pkg/server/node_tombstone_storage.go | 1 - .../profiler/activequeryprofiler_test.go | 1 - pkg/server/settings_cache.go | 2 - pkg/server/telemetry/features.go | 2 - .../tenantsettingswatcher/overrides_store.go | 1 - pkg/settings/values.go | 2 - pkg/spanconfig/spanconfigjob/job.go | 1 - .../spanconfigkvsubscriber/datadriven_test.go | 2 - .../spanconfigkvsubscriber/kvsubscriber.go | 2 - .../spanconfigmanager/manager_test.go | 4 - .../spanconfigreconciler/reconciler.go | 2 - pkg/sql/catalog/lease/lease_internal_test.go | 4 - pkg/sql/catalog/lease/lease_test.go | 15 - pkg/sql/catalog/lease/name_cache.go | 2 - pkg/sql/colfetcher/colbatch_direct_scan.go | 2 - pkg/sql/colfetcher/colbatch_scan.go | 1 - pkg/sql/colfetcher/index_join.go | 1 - pkg/sql/colflow/routers.go | 1 - pkg/sql/colflow/routers_test.go | 1 - pkg/sql/colflow/stats.go | 2 - pkg/sql/colflow/vectorized_flow.go | 1 - pkg/sql/conn_executor_exec.go | 1 - .../concurrent_buffer_guard_test.go | 1 - pkg/sql/contention/event_store.go | 1 - pkg/sql/distsql_running_test.go | 1 - pkg/sql/flowinfra/flow_registry.go | 8 - pkg/sql/flowinfra/flow_registry_test.go | 2 - pkg/sql/idxusage/local_idx_usage_stats.go | 5 - pkg/sql/importer/exportcsv_test.go | 2 - pkg/sql/importer/import_processor.go | 3 - pkg/sql/importer/import_processor_planning.go | 3 - pkg/sql/importer/import_processor_test.go | 3 - pkg/sql/pgwire/hba_conf.go | 1 - pkg/sql/pgwire/server.go | 3 - pkg/sql/rename_test.go | 1 - pkg/sql/rowexec/indexbackfiller.go | 2 - pkg/sql/rowexec/processors_test.go | 6 - pkg/sql/rowflow/routers.go | 8 - pkg/sql/rowflow/routers_test.go | 2 - pkg/sql/schema_changer_test.go | 2 - pkg/sql/schemachanger/corpus/corpus.go | 1 - pkg/sql/sem/builtins/builtins.go | 2 - pkg/sql/sessiondata/sequence_state.go | 2 - pkg/sql/sqlliveness/slinstance/slinstance.go | 2 - .../sqlstats/ssmemstorage/ss_mem_iterator.go | 2 - .../stmtdiagnostics/statement_diagnostics.go | 2 - pkg/sql/tests/rsg_test.go | 3 - pkg/sql/type_change_test.go | 4 - pkg/storage/metamorphic/deck.go | 1 - pkg/storage/pebble.go | 5 - pkg/testutils/distsqlutils/row_buffer.go | 1 - pkg/testutils/lint/lint_test.go | 1 + .../lint/passes/deferunlockcheck/BUILD.bazel | 2 + .../deferunlockcheck/deferunlockcheck.go | 283 +++++++++++++++++- .../deferunlockcheck/testdata/src/a/a.go | 166 ++++++++-- pkg/testutils/net.go | 5 - pkg/testutils/storageutils/mocking.go | 1 - pkg/testutils/testfixtures/test_fixtures.go | 2 - pkg/util/admission/work_queue.go | 9 - pkg/util/admission/work_queue_test.go | 1 - pkg/util/circuit/circuitbreaker.go | 2 - pkg/util/every_n.go | 1 - pkg/util/goschedstats/runnable.go | 1 - pkg/util/hlc/hlc.go | 4 - pkg/util/interval/btree_based_interval.go | 3 - pkg/util/log/buffered_sink.go | 6 + pkg/util/log/buffered_sink_closer_test.go | 3 - pkg/util/log/channels.go | 1 - pkg/util/log/channels_test.go | 4 - pkg/util/log/clog.go | 1 - pkg/util/log/clog_test.go | 3 - pkg/util/log/exit_override.go | 3 - pkg/util/log/file.go | 1 - pkg/util/log/file_api.go | 2 - pkg/util/log/file_log_gc.go | 1 - pkg/util/log/file_log_gc_test.go | 1 - pkg/util/log/flags.go | 4 - pkg/util/log/log_flush.go | 1 - pkg/util/log/registry.go | 4 - pkg/util/log/test_log_scope.go | 7 - pkg/util/log/trace.go | 1 - pkg/util/log/vmodule.go | 1 - pkg/util/metric/hdrhistogram.go | 1 - pkg/util/metric/metric.go | 1 - pkg/util/mon/bytes_usage.go | 2 - pkg/util/randutil/rand.go | 3 - pkg/util/schedulerlatency/sampler.go | 2 - pkg/util/startup/retry.go | 1 - pkg/util/stop/stopper.go | 2 - pkg/util/stop/stopper_test.go | 3 - pkg/util/syncutil/int_map.go | 1 - pkg/util/syncutil/int_map_reference_test.go | 8 - pkg/util/syncutil/mutex_sync_race.go | 7 +- pkg/util/syncutil/mutex_sync_race_test.go | 5 - .../syncutil/singleflight/singleflight.go | 6 - pkg/util/timeutil/pgdate/zone_cache.go | 4 - pkg/util/tracing/crdbspan.go | 4 - pkg/util/tracing/tracer.go | 3 - pkg/util/ulid/ulid_test.go | 1 - pkg/workload/histogram/histogram.go | 4 - pkg/workload/schemachange/deck.go | 1 - pkg/workload/tpcc/new_order.go | 2 - pkg/workload/tpcc/payment.go | 1 - pkg/workload/ycsb/zipfgenerator_test.go | 2 - 285 files changed, 573 insertions(+), 999 deletions(-) diff --git a/build/bazelutil/nogo_config.json b/build/bazelutil/nogo_config.json index 0789c6756e70..2ed44d8ee2b6 100644 --- a/build/bazelutil/nogo_config.json +++ b/build/bazelutil/nogo_config.json @@ -30,89 +30,162 @@ }, "deferunlockcheck": { "exclude_files": { - "pkg/ccl/backupccl/restore_progress.go": "unlock >10 lines away from lock", - "pkg/ccl/changefeedccl/cdctest/mock_webhook_sink.go": "unlock >10 lines away from lock", - "pkg/ccl/changefeedccl/schemafeed/schema_feed.go": "unlock >10 lines away from lock", - "pkg/ccl/changefeedccl/sink_kafka.go": "unlock >10 lines away from lock", - "pkg/ccl/multitenantccl/tenantcostclient/tenant_side.go": "unlock >10 lines away from lock", - "pkg/cmd/generate-spatial-ref-sys/main.go": "unlock >10 lines away from lock", - "pkg/cmd/roachtest/cluster.go": "unlock >10 lines away from lock", - "pkg/cmd/roachtest/test_runner.go": "unlock >10 lines away from lock", - "pkg/cmd/smithtest/main.go": "unlock >10 lines away from lock", - "pkg/gossip/client.go": "unlock >10 lines away from lock", - "pkg/gossip/gossip.go": "unlock >10 lines away from lock", - "pkg/gossip/server.go": "unlock >10 lines away from lock", - "pkg/gossip/storage_test.go": "unlock >10 lines away from lock", - "pkg/internal/sqlsmith/bulkio.go": "unlock >10 lines away from lock", - "pkg/kv/kvclient/kvcoord/txn_coord_sender.go": "unlock >10 lines away from lock", - "pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater_test.go": "unlock >10 lines away from lock", - "pkg/kv/kvclient/kvcoord/dist_sender_ambiguous_test.go": "unlock >10 lines away from lock", - "pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater.go": "unlock >10 lines away from lock", - "pkg/kv/kvclient/kvcoord/txn_lock_gatekeeper.go": "unlock >10 lines away from lock", - "pkg/kv/kvclient/kvstreamer/results_buffer_test.go": "unlock >10 lines away from lock", - "pkg/kv/kvclient/kvstreamer/streamer.go": "unlock >10 lines away from lock", - "pkg/kv/kvclient/rangecache/range_cache.go": "unlock >10 lines away from lock", - "pkg/kv/kvserver/client_merge_test.go": "unlock >10 lines away from lock", - "pkg/kv/kvserver/closedts/sidetransport/debug.go": "unlock >10 lines away from lock", - "pkg/kv/kvserver/closedts/sidetransport/sender.go": "unlock >10 lines away from lock", - "pkg/kv/kvserver/closedts/tracker/tracker_test.go": "unlock >10 lines away from lock", - "pkg/kv/kvserver/concurrency/lock_table.go": "unlock >10 lines away from lock", - "pkg/kv/kvserver/intentresolver/intent_resolver.go": "unlock >10 lines away from lock", - "pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle": "unlock >10 lines away from lock", - "pkg/kv/kvserver/liveness/cache.go": "unlock >10 lines away from lock", - "pkg/kv/kvserver/metrics.go": "unlock >10 lines away from lock", - "pkg/kv/kvserver/queue.go": "unlock >10 lines away from lock", - "pkg/kv/kvserver/raft_log_queue.go": "unlock >10 lines away from lock", - "pkg/kv/kvserver/raft_log_truncator.go": "unlock >10 lines away from lock", - "pkg/kv/kvserver/raftentry/cache.go": "unlock >10 lines away from lock", - "pkg/kv/kvserver/rangefeed/registry.go": "unlock >10 lines away from lock", - "pkg/kv/kvserver/replica.go": "unlock >10 lines away from lock", - "pkg/kv/kvserver/replica_app_batch.go": "unlock >10 lines away from lock", - "pkg/kv/kvserver/replica_metrics.go": "unlock >10 lines away from lock", - "pkg/kv/kvserver/replica_proposal_buf.go": "unlock >10 lines away from lock", - "pkg/kv/kvserver/replica_raft.go": "unlock >10 lines away from lock", - "pkg/kv/kvserver/replica_raftstorage.go": "unlock >10 lines away from lock", - "pkg/kv/kvserver/replica_rangefeed.go": "unlock >10 lines away from lock", - "pkg/kv/kvserver/replica_test.go": "unlock >10 lines away from lock", - "pkg/kv/kvserver/replica_write.go": "unlock >10 lines away from lock", - "pkg/kv/kvserver/scheduler.go": "unlock >10 lines away from lock", - "pkg/kv/kvserver/store.go": "unlock >10 lines away from lock", - "pkg/kv/kvserver/store_test.go": "unlock >10 lines away from lock", - "pkg/kv/kvserver/store_create_replica.go": "unlock >10 lines away from lock", - "pkg/kv/kvserver/store_gossip.go": "unlock >10 lines away from lock", - "pkg/kv/kvserver/store_raft.go": "unlock >10 lines away from lock", - "pkg/kv/kvserver/store_remove_replica.go": "unlock >10 lines away from lock", - "pkg/kv/kvserver/txnwait/queue.go": "unlock >10 lines away from lock", - "pkg/obs/event_exporter.go": "unlock >10 lines away from lock", - "pkg/rpc/clock_offset.go": "unlock >10 lines away from lock", - "pkg/server/status/recorder_test.go": "unlock >10 lines away from lock", - "pkg/server/telemetry/features.go": "unlock >10 lines away from lock", - "pkg/sql/catalog/lease/lease_test.go": "unlock >10 lines away from lock", - "pkg/sql/colflow/routers.go": "unlock >10 lines away from lock", - "pkg/sql/contention/contentionutils/concurrent_buffer_guard.go": "unlock >10 lines away from lock", - "pkg/sql/distsql_running.go": "unlock >10 lines away from lock", - "pkg/sql/distsql_running_test.go": "unlock >10 lines away from lock", - "pkg/sql/flowinfra/flow_registry.go": "unlock >10 lines away from lock", - "pkg/sql/idxusage/local_idx_usage_stats.go": "unlock >10 lines away from lock", - "pkg/sql/pgwire/server.go": "unlock >10 lines away from lock", - "pkg/sql/rowflow/routers.go": "unlock >10 lines away from lock", - "pkg/sql/schema_changer_test.go": "unlock >10 lines away from lock", - "pkg/sql/stats/stats_cache.go": "unlock >10 lines away from lock", - "pkg/sql/tests/rsg_test.go": "unlock >10 lines away from lock", - "pkg/sql/txn_restart_test.go": "unlock >10 lines away from lock", - "pkg/testutils/lint/passes/deferunlockcheck/testdata/src/a/a.go": "test file for the linter", - "pkg/testutils/storageutils/mocking.go": "unlock >10 lines away from lock", - "pkg/util/admission/work_queue.go": "unlock >10 lines away from lock", - "pkg/util/circuit/circuitbreaker.go": "unlock >10 lines away from lock", - "pkg/util/metric/hdrhistogram.go": "unlock >10 lines away from lock", - "pkg/util/mon/bytes_usage.go": "unlock >10 lines away from lock", - "pkg/util/syncutil/int_map.go": "unlock >10 lines away from lock", - "pkg/util/syncutil/singleflight/singleflight": "unlock >10 lines away from lock", - "pkg/util/tracing/crdbspan.go": "unlock >10 lines away from lock", - "pkg/util/tracing/span.go": "unlock >10 lines away from lock", - "pkg/workload/histogram/histogram.go": "unlock >10 lines away from lock", - "pkg/workload/schemachange/schemachange.go": "unlock >10 lines away from lock", - "pkg/workload/tpcc/generate.go": "unlock >10 lines away from lock" + "pkg/.*\\.eg\\.go$": "generated code", + ".*\\.pb\\.go$": "generated code", + ".*\\.pb\\.gw\\.go$": "generated code", + "pkg/.*_generated\\.go$": "generated code", + "_test\\.go$": "tests", + "pkg/ccl/backupccl/backupinfo/manifest_handling.go": "flagged by linter, should be evaluated", + "pkg/ccl/backupccl/restore_job.go": "flagged by linter, should be evaluated", + "pkg/ccl/backupccl/restore_progress.go": "flagged by linter, should be evaluated", + "pkg/ccl/changefeedccl/cdctest/mock_webhook_sink.go": "flagged by linter, should be evaluated", + "pkg/ccl/changefeedccl/changefeed_processors.go": "flagged by linter, should be evaluated", + "pkg/ccl/changefeedccl/event_processing.go": "flagged by linter, should be evaluated", + "pkg/ccl/changefeedccl/kvevent/blocking_buffer.go": "flagged by linter, should be evaluated", + "pkg/ccl/changefeedccl/metrics.go": "flagged by linter, should be evaluated", + "pkg/ccl/changefeedccl/schemafeed/schema_feed.go": "flagged by linter, should be evaluated", + "pkg/ccl/changefeedccl/schema_registry.go": "flagged by linter, should be evaluated", + "pkg/ccl/changefeedccl/sink_kafka.go": "flagged by linter, should be evaluated", + "pkg/ccl/changefeedccl/sink_pubsub_v2.go": "flagged by linter, should be evaluated", + "pkg/ccl/multitenantccl/tenantcostclient/tenant_side.go": "flagged by linter, should be evaluated", + "pkg/ccl/sqlproxyccl/acl/watcher.go": "flagged by linter, should be evaluated", + "pkg/ccl/sqlproxyccl/tenantdirsvr/test_directory_svr.go": "flagged by linter, should be evaluated", + "pkg/ccl/streamingccl/streamclient/random_stream_client.go": "flagged by linter, should be evaluated", + "pkg/cli/clisqlexec/format_csv_tsv.go": "flagged by linter, should be evaluated", + "pkg/cli/clisqlshell/sql.go": "flagged by linter, should be evaluated", + "pkg/cli/syncbench/syncbench.go": "flagged by linter, should be evaluated", + "pkg/cmd/roachtest/cluster.go": "flagged by linter, should be evaluated", + "pkg/cmd/roachtest/test_runner.go": "flagged by linter, should be evaluated", + "pkg/gossip/client.go": "flagged by linter, should be evaluated", + "pkg/gossip/gossip.go": "flagged by linter, should be evaluated", + "pkg/config/system.go": "flagged by linter, should be evaluated", + "pkg/gossip/infostore.go": "flagged by linter, should be evaluated", + "pkg/gossip/server.go": "flagged by linter, should be evaluated", + "pkg/internal/rsg/rsg.go": "flagged by linter, should be evaluated", + "pkg/internal/sqlsmith/bulkio.go": "flagged by linter, should be evaluated", + "pkg/internal/sqlsmith/relational.go": "flagged by linter, should be evaluated", + "pkg/internal/sqlsmith/sampler.go": "flagged by linter, should be evaluated", + "pkg/internal/sqlsmith/scalar.go": "flagged by linter, should be evaluated", + "pkg/jobs/adopt.go": "flagged by linter, should be evaluated", + "pkg/jobs/job_scheduler.go": "flagged by linter, should be evaluated", + "pkg/jobs/progress.go": "flagged by linter, should be evaluated", + "pkg/kv/bulk/buffering_adder.go": "flagged by linter, should be evaluated", + "pkg/kv/bulk/sst_batcher.go": "flagged by linter, should be evaluated", + "pkg/kv/kvclient/kvcoord/dist_sender_mux_rangefeed.go": "flagged by linter, should be evaluated", + "pkg/kv/kvclient/kvcoord/txn_coord_sender.go": "flagged by linter, should be evaluated", + "pkg/kv/kvclient/kvstreamer/streamer.go": "flagged by linter, should be evaluated", + "pkg/kv/kvclient/kvtenant/connector.go": "flagged by linter, should be evaluated", + "pkg/kv/kvclient/rangecache/range_cache.go": "flagged by linter, should be evaluated", + "pkg/kv/kvclient/rangefeed/rangefeedcache/watcher.go": "flagged by linter, should be evaluated", + "pkg/kv/kvserver/allocator/allocatorimpl/allocator_scorer.go": "flagged by linter, should be evaluated", + "pkg/kv/kvserver/allocator/storepool/store_pool.go": "flagged by linter, should be evaluated", + "pkg/kv/kvserver/closedts/sidetransport/debug.go": "flagged by linter, should be evaluated", + "pkg/kv/kvserver/closedts/sidetransport/receiver.go": "flagged by linter, should be evaluated", + "pkg/kv/kvserver/closedts/sidetransport/sender.go": "flagged by linter, should be evaluated", + "pkg/kv/kvserver/concurrency/lock_table.go": "flagged by linter, should be evaluated", + "pkg/kv/kvserver/concurrency/lock_table_waiter.go": "flagged by linter, should be evaluated", + "pkg/kv/kvserver/intentresolver/intent_resolver.go": "flagged by linter, should be evaluated", + "pkg/kv/kvserver/liveness/liveness.go": "flagged by linter, should be evaluated", + "pkg/kv/kvserver/kvflowcontrol/kvflowcontroller/kvflowcontroller.go": "flagged by linter, should be evaluated", + "pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle.go": "flagged by linter, should be evaluated", + "pkg/kv/kvserver/liveness/cache.go": "flagged by linter, should be evaluated", + "pkg/kv/kvserver/metrics.go": "flagged by linter, should be evaluated", + "pkg/kv/kvserver/queue.go": "flagged by linter, should be evaluated", + "pkg/kv/kvserver/raftentry/cache.go": "flagged by linter, should be evaluated", + "pkg/kv/kvserver/raft_log_queue.go": "flagged by linter, should be evaluated", + "pkg/kv/kvserver/raft_log_truncator.go": "flagged by linter, should be evaluated", + "pkg/kv/kvserver/raft_transport.go": "flagged by linter, should be evaluated", + "pkg/kv/kvserver/rangefeed/budget.go": "flagged by linter, should be evaluated", + "pkg/kv/kvserver/rangefeed/scheduler.go": "flagged by linter, should be evaluated", + "pkg/kv/kvserver/replica.go": "flagged by linter, should be evaluated", + "pkg/kv/kvserver/replica_app_batch.go": "flagged by linter, should be evaluated", + "pkg/kv/kvserver/replica_raft.go": "flagged by linter, should be evaluated", + "pkg/kv/kvserver/replica_raft_quiesce.go": "flagged by linter, should be evaluated", + "pkg/kv/kvserver/replica_raftstorage.go": "flagged by linter, should be evaluated", + "pkg/kv/kvserver/replica_rangefeed.go": "flagged by linter, should be evaluated", + "pkg/kv/kvserver/replica_write.go": "flagged by linter, should be evaluated", + "pkg/kv/kvserver/scheduler.go": "flagged by linter, should be evaluated", + "pkg/kv/kvserver/spanlatch/manager.go": "flagged by linter, should be evaluated", + "pkg/kv/kvserver/store.go": "flagged by linter, should be evaluated", + "pkg/kv/kvserver/store_create_replica.go": "flagged by linter, should be evaluated", + "pkg/kv/kvserver/store_gossip.go": "flagged by linter, should be evaluated", + "pkg/kv/kvserver/store_raft.go": "flagged by linter, should be evaluated", + "pkg/kv/kvserver/store_remove_replica.go": "flagged by linter, should be evaluated", + "pkg/kv/kvserver/txnwait/queue.go": "flagged by linter, should be evaluated", + "pkg/kv/kvnemesis/applier.go": "flagged by linter, should be evaluated", + "pkg/kv/kvnemesis/watcher.go": "flagged by linter, should be evaluated", + "pkg/kv/txn.go": "flagged by linter, should be evaluated", + "pkg/obs/event_exporter.go": "flagged by linter, should be evaluated", + "pkg/roachprod/install/cluster_synced.go": "flagged by linter, should be evaluated", + "pkg/roachprod/install/services.go": "flagged by linter, should be evaluated", + "pkg/roachprod/roachprod.go": "flagged by linter, should be evaluated", + "pkg/roachprod/vm/aws/aws.go": "flagged by linter, should be evaluated", + "pkg/roachprod/vm/azure/azure.go": "flagged by linter, should be evaluated", + "pkg/rpc/clock_offset.go": "flagged by linter, should be evaluated", + "pkg/rpc/peer.go": "flagged by linter, should be evaluated", + "pkg/settings/values.go": "flagged by linter, should be evaluated", + "pkg/server/node.go": "flagged by linter, should be evaluated", + "pkg/server/settings_cache.go": "flagged by linter, should be evaluated", + "pkg/server/telemetry/features": "flagged by linter, should be evaluated", + "pkg/spanconfig/spanconfigjob/job.go": "flagged by linter, should be evaluated", + "pkg/spanconfig/spanconfigkvsubscriber/kvsubscriber.go": "flagged by linter, should be evaluated", + "pkg/sql/catalog/lease/name_cache.go": "flagged by linter, should be evaluated", + "pkg/sql/colfetcher/colbatch_direct_scan.go": "flagged by linter, should be evaluated", + "pkg/sql/colfetcher/colbatch_scan.go": "flagged by linter, should be evaluated", + "pkg/sql/colfetcher/index_join.go": "flagged by linter, should be evaluated", + "pkg/sql/colflow/routers.go": "flagged by linter, should be evaluated", + "pkg/sql/colflow/stats.go": "flagged by linter, should be evaluated", + "pkg/sql/contention/contentionutils/concurrent_buffer_guard.go": "flagged by linter, should be evaluated", + "pkg/sql/contention/event_store.go": "flagged by linter, should be evaluated", + "pkg/sql/distsql_running.go": "flagged by linter, should be evaluated", + "pkg/sql/flowinfra/flow_registry.go": "flagged by linter, should be evaluated", + "pkg/sql/idxusage/local_idx_usage_stats.go": "flagged by linter, should be evaluated", + "pkg/sql/importer/import_processor.go": "flagged by linter, should be evaluated", + "pkg/sql/importer/import_processor_planning.go": "flagged by linter, should be evaluated", + "pkg/sql/pgwire/server.go": "flagged by linter, should be evaluated", + "pkg/sql/rowexec/indexbackfiller.go": "flagged by linter, should be evaluated", + "pkg/sql/rowflow/routers.go": "flagged by linter, should be evaluated", + "pkg/sql/schemachanger/corpus/corpus.go": "flagged by linter, should be evaluated", + "pkg/sql/sem/builtins/builtins.go": "flagged by linter, should be evaluated", + "pkg/sql/sqlliveness/slinstance/slinstance.go": "flagged by linter, should be evaluated", + "pkg/sql/sqlstats/ssmemstorage/ss_mem_iterator.go": "flagged by linter, should be evaluated", + "pkg/sql/stats/stats_cache.go": "flagged by linter, should be evaluated", + "pkg/sql/stmtdiagnostics/statement_diagnostics.go": "flagged by linter, should be evaluated", + "pkg/storage/metamorphic/deck.go": "flagged by linter, should be evaluated", + "pkg/storage/pebble.go": "flagged by linter, should be evaluated", + "pkg/util/admission/work_queue.go": "flagged by linter, should be evaluated", + "pkg/util/circuit/circuitbreaker.go": "flagged by linter, should be evaluated", + "pkg/util/every_n.go": "flagged by linter, should be evaluated", + "pkg/util/hlc/hlc.go": "flagged by linter, should be evaluated", + "pkg/util/interval/btree_based_interval.go": "flagged by linter, should be evaluated", + "pkg/util/log/buffered_sink.go": "flagged by linter, should be evaluated", + "pkg/util/log/channels.go": "flagged by linter, should be evaluated", + "pkg/util/log/clog.go": "flagged by linter, should be evaluated", + "pkg/util/log/file.go": "flagged by linter, should be evaluated", + "pkg/util/log/flags.go": "flagged by linter, should be evaluated", + "pkg/util/log/registry.go": "flagged by linter, should be evaluated", + "pkg/util/log/trace.go": "flagged by linter, should be evaluated", + "pkg/util/log/vmodule.go": "flagged by linter, should be evaluated", + "pkg/util/metric/hdrhistogram.go": "flagged by linter, should be evaluated", + "pkg/util/metric/metric.go": "flagged by linter, should be evaluated", + "pkg/util/mon/bytes_usage.go": "flagged by linter, should be evaluated", + "pkg/util/randutil/rand.go": "flagged by linter, should be evaluated", + "pkg/util/schedulerlatency/sampler.go": "flagged by linter, should be evaluated", + "pkg/util/stop/stopper.go": "flagged by linter, should be evaluated", + "pkg/util/startup/retry.go": "flagged by linter, should be evaluated", + "pkg/util/syncutil/int_map.go": "flagged by linter, should be evaluated", + "pkg/util/syncutil/mutex_sync_race.go": "flagged by linter, should be evaluated", + "pkg/util/syncutil/singleflight/singleflight.go": "flagged by linter, should be evaluated", + "pkg/util/tracing/crdbspan.go": "flagged by linter, should be evaluated", + "pkg/util/tracing/span.go": "flagged by linter, should be evaluated", + "pkg/util/tracing/tracer.go": "flagged by linter, should be evaluated", + "pkg/testutils/net.go": "flagged by linter, should be evaluated", + "pkg/testutils/storageutils/mocking.go": "flagged by linter, should be evaluated", + "pkg/testutils/testfixtures/test_fixtures.go": "flagged by linter, should be evaluated", + "pkg/workload/histogram/histogram.go": "flagged by linter, should be evaluated", + "pkg/workload/schemachange/deck.go": "flagged by linter, should be evaluated", + "pkg/workload/schemachange/schemachange.go": "flagged by linter, should be evaluated", + "pkg/workload/tpcc/generate.go": "flagged by linter, should be evaluated" }, "only_files": { "cockroach/pkg/.*$": "first-party code", diff --git a/pkg/acceptance/localcluster/cluster.go b/pkg/acceptance/localcluster/cluster.go index cbe121e9347f..b535b3e5f341 100644 --- a/pkg/acceptance/localcluster/cluster.go +++ b/pkg/acceptance/localcluster/cluster.go @@ -475,7 +475,6 @@ func (n *Node) Alive() bool { func (n *Node) StatusClient(ctx context.Context) serverpb.StatusClient { n.Lock() existingClient := n.statusClient - // nolint:deferunlock n.Unlock() if existingClient != nil { @@ -589,7 +588,6 @@ func (n *Node) startAsyncInnerLocked(ctx context.Context, joins ...string) error _ = errors.As(waitErr, &execErr) n.Lock() n.setNotRunningLocked(execErr) - // nolint:deferunlock n.Unlock() }(n.cmd) @@ -697,7 +695,6 @@ func (n *Node) waitUntilLive(dur time.Duration) error { if n.cmd != nil { pid = n.cmd.Process.Pid } - // nolint:deferunlock n.Unlock() if pid == 0 { log.Info(ctx, "process already quit") @@ -720,14 +717,12 @@ func (n *Node) waitUntilLive(dur time.Duration) error { if n.Cfg.RPCPort == 0 { n.Lock() n.rpcPort = pgURL.Port() - // nolint:deferunlock n.Unlock() } pgURL.Path = n.Cfg.DB n.Lock() n.pgURL = pgURL.String() - // nolint:deferunlock n.Unlock() var uiURL *url.URL @@ -744,7 +739,6 @@ func (n *Node) waitUntilLive(dur time.Duration) error { // http port is required but isn't initialized yet. n.Lock() n.db = makeDB(n.pgURL, n.Cfg.NumWorkers, n.Cfg.DB) - // nolint:deferunlock n.Unlock() { @@ -775,7 +769,6 @@ func (n *Node) Kill() { for ok := false; !ok; { n.Lock() ok = n.cmd == nil - // nolint:deferunlock n.Unlock() } } @@ -811,7 +804,6 @@ func (n *Node) Signal(s os.Signal) { func (n *Node) Wait() *exec.ExitError { n.Lock() ch := n.notRunning - // nolint:deferunlock n.Unlock() if ch == nil { log.Warning(context.Background(), "(*Node).Wait called when node was not running") diff --git a/pkg/ccl/auditloggingccl/audit_log_config.go b/pkg/ccl/auditloggingccl/audit_log_config.go index 20075715865f..732e6e6a271b 100644 --- a/pkg/ccl/auditloggingccl/audit_log_config.go +++ b/pkg/ccl/auditloggingccl/audit_log_config.go @@ -85,7 +85,6 @@ func UpdateAuditConfigOnChange( } acl.Lock() acl.Config = config - // nolint:deferunlock acl.Unlock() } diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index bf463bd63061..d36bcf3420ca 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -2593,11 +2593,9 @@ func TestBackupRestoreDuringUserDefinedTypeChange(t *testing.T) { if numTypeChangesStarted == len(tc.queries) { close(typeChangesStarted) } - // nolint:deferunlock mu.Unlock() <-waitForBackup } else { - // nolint:deferunlock mu.Unlock() } return nil @@ -2626,7 +2624,6 @@ CREATE TYPE d.greeting AS ENUM ('hello', 'howdy', 'hi'); if numTypeChangesFinished == totalQueries { close(typeChangesFinished) } - // nolint:deferunlock mu.Unlock() }(query, len(tc.queries)) } @@ -6208,7 +6205,6 @@ func TestRestoreErrorPropagates(t *testing.T) { jobsTableKey.Lock() jobsTableKey.key = tc.ApplicationLayer(0).Codec().TablePrefix(uint32(systemschema.JobsTable.GetID())) - // nolint:deferunlock jobsTableKey.Unlock() runner.Exec(t, `SET CLUSTER SETTING jobs.metrics.interval.poll = '30s'`) @@ -6391,7 +6387,6 @@ INSERT INTO foo.bar VALUES (110), (210), (310), (410), (510)`) startingSpan := mkSpan(id1, "/Tenant/10/Table/:id/1", "/Tenant/10/Table/:id/2") mu.Lock() require.Equal(t, []string{startingSpan.String()}, mu.exportRequestSpans) - // nolint:deferunlock mu.Unlock() resetStateVars() @@ -6402,7 +6397,6 @@ INSERT INTO foo.bar VALUES (110), (210), (310), (410), (510)`) resumeSpan := mkSpan(id1, "/Tenant/10/Table/:id/1/510/0", "/Tenant/10/Table/:id/2") mu.Lock() require.Equal(t, []string{startingSpan.String(), resumeSpan.String()}, mu.exportRequestSpans) - // nolint:deferunlock mu.Unlock() resetStateVars() @@ -6421,7 +6415,6 @@ INSERT INTO foo.bar VALUES (110), (210), (310), (410), (510)`) } mu.Lock() require.Equal(t, expected, mu.exportRequestSpans) - // nolint:deferunlock mu.Unlock() resetStateVars() @@ -6455,7 +6448,6 @@ INSERT INTO baz.bar VALUES (110, 'a'), (210, 'b'), (310, 'c'), (410, 'd'), (510, } mu.Lock() require.Equal(t, expected, mu.exportRequestSpans) - // nolint:deferunlock mu.Unlock() resetStateVars() diff --git a/pkg/ccl/backupccl/backupinfo/manifest_handling.go b/pkg/ccl/backupccl/backupinfo/manifest_handling.go index 99cbd557fe03..3edc5bc98140 100644 --- a/pkg/ccl/backupccl/backupinfo/manifest_handling.go +++ b/pkg/ccl/backupccl/backupinfo/manifest_handling.go @@ -1541,7 +1541,6 @@ func GetBackupManifests( manifests[i] = desc } subMem.Shrink(ctx, size) - // nolint:deferunlock memMu.Unlock() return err diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index 0241f817b2c7..58c093bcee31 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -318,7 +318,6 @@ func restore( introducedSpanFrontier, targetRestoreSpanSize.Get(&execCtx.ExecCfg().Settings.SV), progressTracker.useFrontier) - // nolint:deferunlock progressTracker.mu.Unlock() if err != nil { return roachpb.RowCount{}, err @@ -2154,7 +2153,6 @@ func insertStats( mu.completedBatches++ remainingBatches := totalNumBatches - mu.completedBatches completedBatches := mu.completedBatches - // nolint:deferunlock mu.Unlock() if insertStatsProgress.ShouldLog() { logStatsProgress(remainingBatches, completedBatches) diff --git a/pkg/ccl/backupccl/tenant_backup_nemesis_test.go b/pkg/ccl/backupccl/tenant_backup_nemesis_test.go index c69985bc440f..a66eb817d145 100644 --- a/pkg/ccl/backupccl/tenant_backup_nemesis_test.go +++ b/pkg/ccl/backupccl/tenant_backup_nemesis_test.go @@ -428,7 +428,6 @@ func (r *randomBackupNemesis) Stop() { func (r *randomBackupNemesis) TablesToCheck() []string { r.mu.Lock() ret := append([]string(nil), r.mu.tablesToCheck...) - // nolint:deferunlock r.mu.Unlock() return ret } @@ -436,7 +435,6 @@ func (r *randomBackupNemesis) TablesToCheck() []string { func (r *randomBackupNemesis) addTable(name string) { r.mu.Lock() r.mu.tablesToCheck = append(r.mu.tablesToCheck, name) - // nolint:deferunlock r.mu.Unlock() } diff --git a/pkg/ccl/changefeedccl/cdctest/schema_registry.go b/pkg/ccl/changefeedccl/cdctest/schema_registry.go index d15e83ee7a75..ebb59f936095 100644 --- a/pkg/ccl/changefeedccl/cdctest/schema_registry.go +++ b/pkg/ccl/changefeedccl/cdctest/schema_registry.go @@ -207,7 +207,6 @@ func (r *SchemaRegistry) EncodedAvroToNative(b []byte) (interface{}, error) { r.mu.Lock() jsonSchema := r.mu.schemas[id] - // nolint:deferunlock r.mu.Unlock() codec, err := goavro.NewCodec(jsonSchema) if err != nil { diff --git a/pkg/ccl/changefeedccl/changefeed_processors.go b/pkg/ccl/changefeedccl/changefeed_processors.go index 11d53f66bb31..60a9b05f67da 100644 --- a/pkg/ccl/changefeedccl/changefeed_processors.go +++ b/pkg/ccl/changefeedccl/changefeed_processors.go @@ -1187,7 +1187,6 @@ func (cf *changeFrontier) Start(ctx context.Context) { cf.metricsID = cf.metrics.mu.id cf.metrics.mu.id++ sli.RunningCount.Inc(1) - // nolint:deferunlock cf.metrics.mu.Unlock() cf.sliMetricsID = cf.sliMetrics.claimId() @@ -1230,7 +1229,6 @@ func (cf *changeFrontier) closeMetrics() { } delete(cf.metrics.mu.resolved, cf.metricsID) cf.metricsID = -1 - // nolint:deferunlock cf.metrics.mu.Unlock() cf.sliMetrics.closeId(cf.sliMetricsID) @@ -1383,7 +1381,6 @@ func (cf *changeFrontier) forwardFrontier(resolved jobspb.ResolvedSpan) error { if cf.metricsID != -1 { cf.metrics.mu.resolved[cf.metricsID] = newResolved } - // nolint:deferunlock cf.metrics.mu.Unlock() return cf.maybeEmitResolved(newResolved) diff --git a/pkg/ccl/changefeedccl/event_processing.go b/pkg/ccl/changefeedccl/event_processing.go index b17f054c2841..a447cbcc250f 100644 --- a/pkg/ccl/changefeedccl/event_processing.go +++ b/pkg/ccl/changefeedccl/event_processing.go @@ -646,7 +646,6 @@ func (c *parallelEventConsumer) incInFlight() { c.mu.Lock() c.mu.inFlight++ c.metrics.ParallelConsumerInFlightEvents.Update(int64(c.mu.inFlight)) - // nolint:deferunlock c.mu.Unlock() } @@ -654,7 +653,6 @@ func (c *parallelEventConsumer) decInFlight() { c.mu.Lock() c.mu.inFlight-- notifyFlush := c.mu.waiting && c.mu.inFlight == 0 - // nolint:deferunlock c.mu.Unlock() // If someone is waiting on a flush, signal to them @@ -708,7 +706,6 @@ func (c *parallelEventConsumer) Flush(ctx context.Context) error { c.mu.Lock() c.mu.waiting = false c.mu.flushFrontier = c.spanFrontier.Frontier() - // nolint:deferunlock c.mu.Unlock() return nil } diff --git a/pkg/ccl/changefeedccl/kvevent/blocking_buffer.go b/pkg/ccl/changefeedccl/kvevent/blocking_buffer.go index d3bba5058c3e..95eea2ae9d80 100644 --- a/pkg/ccl/changefeedccl/kvevent/blocking_buffer.go +++ b/pkg/ccl/changefeedccl/kvevent/blocking_buffer.go @@ -150,7 +150,6 @@ func (b *blockingBuffer) pop() (e Event, ok bool, err error) { func (b *blockingBuffer) notifyOutOfQuota(canFlush bool) { b.mu.Lock() b.mu.canFlush = canFlush - // nolint:deferunlock b.mu.Unlock() if canFlush { @@ -166,7 +165,6 @@ func (b *blockingBuffer) notifyOutOfQuota(canFlush bool) { func (b *blockingBuffer) producerBlocked() { b.mu.Lock() b.mu.numBlocked++ - // nolint:deferunlock b.mu.Unlock() } @@ -185,7 +183,6 @@ func (b *blockingBuffer) quotaAcquiredAfterWait() { // Clear out canFlush since we know that producers no longer blocked. b.mu.canFlush = false } - // nolint:deferunlock b.mu.Unlock() } diff --git a/pkg/ccl/changefeedccl/metrics.go b/pkg/ccl/changefeedccl/metrics.go index ed762d41a9ff..cde48c442fc9 100644 --- a/pkg/ccl/changefeedccl/metrics.go +++ b/pkg/ccl/changefeedccl/metrics.go @@ -848,7 +848,6 @@ func MakeMetrics(histogramWindow time.Duration) metric.Struct { maxBehind = behind } } - // nolint:deferunlock m.mu.Unlock() return maxBehind.Nanoseconds() }) diff --git a/pkg/ccl/changefeedccl/schema_registry.go b/pkg/ccl/changefeedccl/schema_registry.go index 1fdcde7fe9b9..fb4ad25e4f92 100644 --- a/pkg/ccl/changefeedccl/schema_registry.go +++ b/pkg/ccl/changefeedccl/schema_registry.go @@ -148,7 +148,6 @@ func newConfluentSchemaRegistry( } schemaRegistrySingletons.cachePerEndpoint[baseURL] = src } - // nolint:deferunlock schemaRegistrySingletons.mu.Unlock() s, err := getAndDeleteParams(u) diff --git a/pkg/ccl/changefeedccl/schemafeed/schema_feed.go b/pkg/ccl/changefeedccl/schemafeed/schema_feed.go index b57b2361f74c..74c3cd992824 100644 --- a/pkg/ccl/changefeedccl/schemafeed/schema_feed.go +++ b/pkg/ccl/changefeedccl/schemafeed/schema_feed.go @@ -280,7 +280,6 @@ func (tf *schemaFeed) Run(ctx context.Context) error { func (tf *schemaFeed) primeInitialTableDescs(ctx context.Context) error { tf.mu.Lock() initialTableDescTs := tf.mu.highWater - // nolint:deferunlock tf.mu.Unlock() var initialDescs []catalog.Descriptor @@ -313,7 +312,6 @@ func (tf *schemaFeed) primeInitialTableDescs(ctx context.Context) error { tbl := desc.(catalog.TableDescriptor) tf.mu.typeDeps.ingestTable(tbl) } - // nolint:deferunlock tf.mu.Unlock() return tf.ingestDescriptors(ctx, hlc.Timestamp{}, initialTableDescTs, initialDescs, tf.validateDescriptor) @@ -512,7 +510,6 @@ func (tf *schemaFeed) pauseOrResumePolling( func (tf *schemaFeed) highWater() hlc.Timestamp { tf.mu.Lock() highWater := tf.mu.highWater - // nolint:deferunlock tf.mu.Unlock() return highWater } diff --git a/pkg/ccl/changefeedccl/sink_kafka.go b/pkg/ccl/changefeedccl/sink_kafka.go index 7aa8c39550ce..7834a9b8b11f 100644 --- a/pkg/ccl/changefeedccl/sink_kafka.go +++ b/pkg/ccl/changefeedccl/sink_kafka.go @@ -52,7 +52,6 @@ func (l *maybeLocker) Lock() { } func (l *maybeLocker) Unlock() { if l.locked { - // nolint:deferunlock l.wrapped.Unlock() l.locked = false } @@ -422,7 +421,6 @@ func (s *kafkaSink) Flush(ctx context.Context) error { if !immediateFlush { s.mu.flushCh = flushCh } - // nolint:deferunlock s.mu.Unlock() if immediateFlush { @@ -439,7 +437,6 @@ func (s *kafkaSink) Flush(ctx context.Context) error { s.mu.Lock() flushErr := s.mu.flushErr s.mu.flushErr = nil - // nolint:deferunlock s.mu.Unlock() return flushErr } diff --git a/pkg/ccl/changefeedccl/sink_pubsub_v2.go b/pkg/ccl/changefeedccl/sink_pubsub_v2.go index 78bfb6e44b18..1b069b2ff16e 100644 --- a/pkg/ccl/changefeedccl/sink_pubsub_v2.go +++ b/pkg/ccl/changefeedccl/sink_pubsub_v2.go @@ -142,11 +142,9 @@ func (sc *pubsubSinkClient) maybeCreateTopic(topic string) error { sc.mu.RLock() _, ok := sc.mu.topicCache[topic] if ok { - // nolint:deferunlock sc.mu.RUnlock() return nil } - // nolint:deferunlock sc.mu.RUnlock() sc.mu.Lock() defer sc.mu.Unlock() diff --git a/pkg/ccl/changefeedccl/sink_test.go b/pkg/ccl/changefeedccl/sink_test.go index 1bbf15aa99f4..c3ec5a021772 100644 --- a/pkg/ccl/changefeedccl/sink_test.go +++ b/pkg/ccl/changefeedccl/sink_test.go @@ -177,7 +177,6 @@ func (p *asyncProducerMock) consume() (cleanup func()) { case m := <-p.inputCh: p.mu.Lock() p.mu.outstanding = append(p.mu.outstanding, m) - // nolint:deferunlock p.mu.Unlock() } } @@ -196,7 +195,6 @@ func (p *asyncProducerMock) acknowledge(n int, ch chan *sarama.ProducerMessage) p.mu.Lock() outstanding = append(outstanding, p.mu.outstanding...) p.mu.outstanding = p.mu.outstanding[:0] - // nolint:deferunlock p.mu.Unlock() for _, m := range outstanding { diff --git a/pkg/ccl/multiregionccl/datadriven_test.go b/pkg/ccl/multiregionccl/datadriven_test.go index 2746497c93b9..c046b463a980 100644 --- a/pkg/ccl/multiregionccl/datadriven_test.go +++ b/pkg/ccl/multiregionccl/datadriven_test.go @@ -249,12 +249,10 @@ SET CLUSTER SETTING kv.allocator.min_lease_transfer_interval = 5m // Setup tracing for the input. mu.Lock() traceStmt = d.Input - // nolint:deferunlock mu.Unlock() defer func() { mu.Lock() traceStmt = "" - // nolint:deferunlock mu.Unlock() }() diff --git a/pkg/ccl/multiregionccl/region_test.go b/pkg/ccl/multiregionccl/region_test.go index f0b9f9c8310e..eafdeaab7b31 100644 --- a/pkg/ccl/multiregionccl/region_test.go +++ b/pkg/ccl/multiregionccl/region_test.go @@ -199,13 +199,11 @@ func TestConcurrentAddDropRegions(t *testing.T) { if firstOp { firstOp = false close(firstOpStarted) - // nolint:deferunlock mu.Unlock() // Don't promote any members before the second operation reaches // the schema changer as well. <-secondOpFinished } else { - // nolint:deferunlock mu.Unlock() } return nil diff --git a/pkg/ccl/multitenantccl/tenantcostclient/tenant_side.go b/pkg/ccl/multitenantccl/tenantcostclient/tenant_side.go index 99326c2b697e..628ee43debdd 100644 --- a/pkg/ccl/multitenantccl/tenantcostclient/tenant_side.go +++ b/pkg/ccl/multitenantccl/tenantcostclient/tenant_side.go @@ -441,7 +441,6 @@ func (c *tenantSideCostController) onTick(ctx context.Context, newTime time.Time c.mu.consumption.PGWireEgressBytes += deltaPGWireEgressBytes c.mu.consumption.RU += float64(ru) newConsumption := c.mu.consumption - // nolint:deferunlock c.mu.Unlock() // Update the average RUs consumed per second, based on the latest stats. @@ -880,7 +879,6 @@ func (c *tenantSideCostController) onExternalIO( if c.shouldAccountForExternalIORUs() { c.mu.consumption.RU += float64(totalRU) } - // nolint:deferunlock c.mu.Unlock() return nil diff --git a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/datadriven_test.go b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/datadriven_test.go index 658d53330465..75af57f0dce7 100644 --- a/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/datadriven_test.go +++ b/pkg/ccl/spanconfigccl/spanconfigsqltranslatorccl/datadriven_test.go @@ -91,7 +91,6 @@ func TestDataDriven(t *testing.T) { for !allowGC { gcWaiter.Wait() } - // nolint:deferunlock gcWaiter.L.Unlock() return nil }, @@ -291,14 +290,12 @@ func TestDataDriven(t *testing.T) { case "block-gc-jobs": gcWaiter.L.Lock() allowGC = false - // nolint:deferunlock gcWaiter.L.Unlock() case "unblock-gc-jobs": gcWaiter.L.Lock() allowGC = true gcWaiter.Signal() - // nolint:deferunlock gcWaiter.L.Unlock() default: t.Fatalf("unknown command: %s", d.Cmd) diff --git a/pkg/ccl/spanconfigccl/spanconfigsqlwatcherccl/sqlwatcher_test.go b/pkg/ccl/spanconfigccl/spanconfigsqlwatcherccl/sqlwatcher_test.go index 98f01faac865..7cfeec097ee0 100644 --- a/pkg/ccl/spanconfigccl/spanconfigsqlwatcherccl/sqlwatcher_test.go +++ b/pkg/ccl/spanconfigccl/spanconfigsqlwatcherccl/sqlwatcher_test.go @@ -358,7 +358,6 @@ func TestSQLWatcherMultiple(t *testing.T) { go f(watcher1Ctx, func() { wg.Done() }, func(ts hlc.Timestamp) { mu.Lock() mu.w1LastCheckpoint = ts - // nolint:deferunlock mu.Unlock() }) @@ -384,7 +383,6 @@ func TestSQLWatcherMultiple(t *testing.T) { go f(watcher2Ctx, func() { wg.Done() }, func(ts hlc.Timestamp) { mu.Lock() mu.w2LastCheckpoint = ts - // nolint:deferunlock mu.Unlock() }) @@ -393,7 +391,6 @@ func TestSQLWatcherMultiple(t *testing.T) { go f(watcher3Ctx, func() { wg.Done() }, func(ts hlc.Timestamp) { mu.Lock() mu.w3LastCheckpoint = ts - // nolint:deferunlock mu.Unlock() }) @@ -621,7 +618,6 @@ func TestWatcherReceivesNoopCheckpoints(t *testing.T) { mu.Lock() mu.lastCheckpoint = ts mu.numCheckpoints++ - // nolint:deferunlock mu.Unlock() }) diff --git a/pkg/ccl/sqlproxyccl/acl/watcher.go b/pkg/ccl/sqlproxyccl/acl/watcher.go index aca6f89b6a3b..fbf90b5ccd19 100644 --- a/pkg/ccl/sqlproxyccl/acl/watcher.go +++ b/pkg/ccl/sqlproxyccl/acl/watcher.go @@ -217,7 +217,6 @@ func (w *Watcher) addAccessController( w.mu.Lock() index := len(w.controllers) w.controllers = append(w.controllers, controller) - // nolint:deferunlock w.mu.Unlock() if next != nil { @@ -240,7 +239,6 @@ func (w *Watcher) updateAccessController( copy := w.listeners.Clone() w.controllers[index] = controller controllers := append([]AccessController(nil), w.controllers...) - // nolint:deferunlock w.mu.Unlock() checkListeners(ctx, copy, controllers) diff --git a/pkg/ccl/sqlproxyccl/balancer/balancer_test.go b/pkg/ccl/sqlproxyccl/balancer/balancer_test.go index 98c869377396..c3e75179523d 100644 --- a/pkg/ccl/sqlproxyccl/balancer/balancer_test.go +++ b/pkg/ccl/sqlproxyccl/balancer/balancer_test.go @@ -774,7 +774,6 @@ func TestBalancer_RebalanceTenant_WithRebalancingDisabled(t *testing.T) { // Queue should be empty, and no additional connections should be moved. b.queue.mu.Lock() queueLen := b.queue.queue.Len() - // nolint:deferunlock b.queue.mu.Unlock() require.Equal(t, 0, queueLen) assertZeroTransfers() @@ -787,7 +786,6 @@ func TestBalancer_RebalanceTenant_WithRebalancingDisabled(t *testing.T) { // Queue should be empty, and no additional connections should be moved. b.queue.mu.Lock() queueLen = b.queue.queue.Len() - // nolint:deferunlock b.queue.mu.Unlock() require.Equal(t, 0, queueLen) assertZeroTransfers() @@ -879,7 +877,6 @@ func TestBalancer_RebalanceTenant_WithDefaultDelay(t *testing.T) { // Queue should be empty, and no additional connections should be moved. b.queue.mu.Lock() queueLen := b.queue.queue.Len() - // nolint:deferunlock b.queue.mu.Unlock() require.Equal(t, 0, queueLen) waitFor(50) diff --git a/pkg/ccl/sqlproxyccl/forwarder_test.go b/pkg/ccl/sqlproxyccl/forwarder_test.go index 5aa18be5da3c..d51c301ef0cf 100644 --- a/pkg/ccl/sqlproxyccl/forwarder_test.go +++ b/pkg/ccl/sqlproxyccl/forwarder_test.go @@ -91,7 +91,6 @@ func TestForward(t *testing.T) { f.mu.Lock() requestProc := f.mu.request - // nolint:deferunlock f.mu.Unlock() initialClock := requestProc.logicalClockFn() @@ -105,7 +104,6 @@ func TestForward(t *testing.T) { requestProc.mu.Lock() require.True(t, requestProc.mu.resumed) - // nolint:deferunlock requestProc.mu.Unlock() // Client writes some pgwire messages. @@ -149,7 +147,6 @@ func TestForward(t *testing.T) { requestProc.mu.Lock() require.Equal(t, byte(pgwirebase.ClientMsgSimpleQuery), requestProc.mu.lastMessageType) require.Equal(t, initialClock+1, requestProc.mu.lastMessageTransferredAt) - // nolint:deferunlock requestProc.mu.Unlock() require.False(t, f.IsIdle()) barrierEnd <- struct{}{} @@ -171,7 +168,6 @@ func TestForward(t *testing.T) { requestProc.mu.Lock() require.Equal(t, byte(pgwirebase.ClientMsgExecute), requestProc.mu.lastMessageType) require.Equal(t, initialClock+2, requestProc.mu.lastMessageTransferredAt) - // nolint:deferunlock requestProc.mu.Unlock() require.False(t, f.IsIdle()) barrierEnd <- struct{}{} @@ -194,7 +190,6 @@ func TestForward(t *testing.T) { requestProc.mu.Lock() require.Equal(t, byte(pgwirebase.ClientMsgClose), requestProc.mu.lastMessageType) require.Equal(t, initialClock+3, requestProc.mu.lastMessageTransferredAt) - // nolint:deferunlock requestProc.mu.Unlock() require.False(t, f.IsIdle()) barrierEnd <- struct{}{} @@ -241,7 +236,6 @@ func TestForward(t *testing.T) { f.mu.Lock() responseProc := f.mu.response - // nolint:deferunlock f.mu.Unlock() initialClock := responseProc.logicalClockFn() @@ -255,7 +249,6 @@ func TestForward(t *testing.T) { responseProc.mu.Lock() require.True(t, responseProc.mu.resumed) - // nolint:deferunlock responseProc.mu.Unlock() // Server writes some pgwire messages. @@ -292,7 +285,6 @@ func TestForward(t *testing.T) { responseProc.mu.Lock() require.Equal(t, byte(pgwirebase.ServerMsgErrorResponse), responseProc.mu.lastMessageType) require.Equal(t, initialClock+1, responseProc.mu.lastMessageTransferredAt) - // nolint:deferunlock responseProc.mu.Unlock() require.False(t, f.IsIdle()) barrierEnd <- struct{}{} @@ -315,7 +307,6 @@ func TestForward(t *testing.T) { responseProc.mu.Lock() require.Equal(t, byte(pgwirebase.ServerMsgReady), responseProc.mu.lastMessageType) require.Equal(t, initialClock+2, responseProc.mu.lastMessageTransferredAt) - // nolint:deferunlock responseProc.mu.Unlock() require.False(t, f.IsIdle()) barrierEnd <- struct{}{} @@ -558,13 +549,11 @@ func TestSuspendResumeProcessor(t *testing.T) { require.EqualError(t, p.resume(ctx), context.Canceled.Error()) p.mu.Lock() require.True(t, p.mu.closed) - // nolint:deferunlock p.mu.Unlock() // Set resumed to true to simulate suspend loop. p.mu.Lock() p.mu.resumed = true - // nolint:deferunlock p.mu.Unlock() require.EqualError(t, p.suspend(ctx), errProcessorClosed.Error()) }) @@ -649,7 +638,6 @@ func TestSuspendResumeProcessor(t *testing.T) { require.False(t, p.mu.resumed) require.False(t, p.mu.inPeek) require.False(t, p.mu.suspendReq) - // nolint:deferunlock p.mu.Unlock() // Suspend a second time. @@ -661,7 +649,6 @@ func TestSuspendResumeProcessor(t *testing.T) { require.False(t, p.mu.resumed) require.False(t, p.mu.inPeek) require.False(t, p.mu.suspendReq) - // nolint:deferunlock p.mu.Unlock() }) diff --git a/pkg/ccl/sqlproxyccl/proxy_handler_test.go b/pkg/ccl/sqlproxyccl/proxy_handler_test.go index 2445161a8471..49f5aefd773e 100644 --- a/pkg/ccl/sqlproxyccl/proxy_handler_test.go +++ b/pkg/ccl/sqlproxyccl/proxy_handler_test.go @@ -1669,7 +1669,6 @@ func TestCancelQuery(t *testing.T) { }) origCancelInfo.mu.RLock() origKey := origCancelInfo.mu.origBackendKeyData.SecretKey - // nolint:deferunlock origCancelInfo.mu.RUnlock() // Advance the time so that rebalancing will occur. timeSource.Advance(2 * time.Minute) @@ -1681,7 +1680,6 @@ func TestCancelQuery(t *testing.T) { } newCancelInfo.mu.RLock() newKey := newCancelInfo.mu.origBackendKeyData.SecretKey - // nolint:deferunlock newCancelInfo.mu.RUnlock() if origKey == newKey { return errors.Newf("expected %d to differ", origKey) diff --git a/pkg/ccl/sqlproxyccl/tenantdirsvr/test_directory_svr.go b/pkg/ccl/sqlproxyccl/tenantdirsvr/test_directory_svr.go index 7e94aefd62a7..d64f2a1349cb 100644 --- a/pkg/ccl/sqlproxyccl/tenantdirsvr/test_directory_svr.go +++ b/pkg/ccl/sqlproxyccl/tenantdirsvr/test_directory_svr.go @@ -176,7 +176,6 @@ func (s *TestDirectoryServer) WatchPods( c := make(chan *tenant.WatchPodsResponse, 10) s.mu.Lock() elem := s.mu.eventListeners.PushBack(c) - // nolint:deferunlock s.mu.Unlock() err := s.stopper.RunTask(context.Background(), "watch-pods-server", func(ctx context.Context) { @@ -191,7 +190,6 @@ func (s *TestDirectoryServer) WatchPods( s.mu.Lock() s.mu.eventListeners.Remove(elem) close(c) - // nolint:deferunlock s.mu.Unlock() break out } @@ -199,7 +197,6 @@ func (s *TestDirectoryServer) WatchPods( s.mu.Lock() s.mu.eventListeners.Remove(elem) close(c) - // nolint:deferunlock s.mu.Unlock() break out } diff --git a/pkg/ccl/sqlproxyccl/tenantdirsvr/test_static_directory_svr.go b/pkg/ccl/sqlproxyccl/tenantdirsvr/test_static_directory_svr.go index dab66f70c58e..9e78f11e91b3 100644 --- a/pkg/ccl/sqlproxyccl/tenantdirsvr/test_static_directory_svr.go +++ b/pkg/ccl/sqlproxyccl/tenantdirsvr/test_static_directory_svr.go @@ -129,7 +129,6 @@ func (d *TestStaticDirectoryServer) WatchPods( ) error { d.process.Lock() stopper := d.process.stopper - // nolint:deferunlock d.process.Unlock() // This cannot happen unless WatchPods was called directly, which we @@ -242,7 +241,6 @@ func (d *TestStaticDirectoryServer) WatchTenants( ) error { d.process.Lock() stopper := d.process.stopper - // nolint:deferunlock d.process.Unlock() // This cannot happen unless WatchTenants was called directly, which we @@ -513,7 +511,6 @@ func (d *TestStaticDirectoryServer) Stop(ctx context.Context) { func (d *TestStaticDirectoryServer) DialerFunc(ctx context.Context, addr string) (net.Conn, error) { d.process.Lock() listener := d.process.ln - // nolint:deferunlock d.process.Unlock() if listener == nil { diff --git a/pkg/ccl/streamingccl/streamclient/client_test.go b/pkg/ccl/streamingccl/streamclient/client_test.go index 54b974d37467..c703bad91411 100644 --- a/pkg/ccl/streamingccl/streamclient/client_test.go +++ b/pkg/ccl/streamingccl/streamclient/client_test.go @@ -221,7 +221,6 @@ func ExampleClient() { case <-ticker.C: ingested.Lock() ts := ingested.ts - // nolint:deferunlock ingested.Unlock() if _, err := client.Heartbeat(ctx, id, ts); err != nil { @@ -235,7 +234,6 @@ func ExampleClient() { defer close(done) ingested.Lock() ts := ingested.ts - // nolint:deferunlock ingested.Unlock() topology, err := client.Plan(ctx, id) @@ -273,7 +271,6 @@ func ExampleClient() { } } ingested.ts.Forward(minTS) - // nolint:deferunlock ingested.Unlock() fmt.Printf("resolved %d\n", minTS.WallTime) default: diff --git a/pkg/ccl/streamingccl/streamclient/random_stream_client.go b/pkg/ccl/streamingccl/streamclient/random_stream_client.go index 2e930a675cef..436507bf805f 100644 --- a/pkg/ccl/streamingccl/streamclient/random_stream_client.go +++ b/pkg/ccl/streamingccl/streamclient/random_stream_client.go @@ -496,7 +496,6 @@ func (m *RandomStreamClient) Subscribe( rng, _ := randutil.NewPseudoRand() m.mu.Lock() reg, err := newRandomEventGenerator(rng, partitionURL, config, m.mu.sstMaker) - // nolint:deferunlock m.mu.Unlock() if err != nil { return nil, err diff --git a/pkg/ccl/streamingccl/streamingest/replication_stream_e2e_test.go b/pkg/ccl/streamingccl/streamingest/replication_stream_e2e_test.go index e67c821c4a5d..797e8dd518c8 100644 --- a/pkg/ccl/streamingccl/streamingest/replication_stream_e2e_test.go +++ b/pkg/ccl/streamingccl/streamingest/replication_stream_e2e_test.go @@ -142,7 +142,6 @@ func TestTenantStreamingJobRetryReset(t *testing.T) { c.WaitUntilReplicatedTime(srcTime, jobspb.JobID(ingestionJobID)) mu.Lock() mu.initialScanComplete = true - // nolint:deferunlock mu.Unlock() jobutils.WaitForJobToPause(t, c.DestSysSQL, jobspb.JobID(ingestionJobID)) @@ -1094,7 +1093,6 @@ func TestTenantStreamingRetryLoadJob(t *testing.T) { // Inject an error to fail the resumer. mu.Lock() knobLoadErr = errors.Newf("test error") - // nolint:deferunlock mu.Unlock() // Resume ingestion. diff --git a/pkg/ccl/streamingccl/streamproducer/replication_stream_test.go b/pkg/ccl/streamingccl/streamproducer/replication_stream_test.go index 656173446622..02113f1b7ba0 100644 --- a/pkg/ccl/streamingccl/streamproducer/replication_stream_test.go +++ b/pkg/ccl/streamingccl/streamproducer/replication_stream_test.go @@ -88,7 +88,6 @@ func (f *pgConnReplicationFeedSource) Close(ctx context.Context) { f.mu.Lock() f.mu.rows.Close() - // nolint:deferunlock f.mu.Unlock() require.NoError(f.t, f.conn.Close(ctx)) } @@ -164,7 +163,6 @@ func (f *pgConnReplicationFeedSource) Error() error { var err error f.mu.Lock() err = f.mu.rows.Err() - // nolint:deferunlock f.mu.Unlock() return err } diff --git a/pkg/cli/clisqlexec/format_csv_tsv.go b/pkg/cli/clisqlexec/format_csv_tsv.go index a16c14d21d6b..58134182f2e6 100644 --- a/pkg/cli/clisqlexec/format_csv_tsv.go +++ b/pkg/cli/clisqlexec/format_csv_tsv.go @@ -48,7 +48,6 @@ func makeCSVReporter(w io.Writer, format TableDisplayFormat) (*csvReporter, func case <-ticker.C: r.mu.Lock() r.mu.csvWriter.Flush() - // nolint:deferunlock r.mu.Unlock() case <-r.stop: return @@ -68,7 +67,6 @@ func (p *csvReporter) describe(w io.Writer, cols []string) error { } else { _ = p.mu.csvWriter.Write(cols) } - // nolint:deferunlock p.mu.Unlock() return nil } @@ -80,7 +78,6 @@ func (p *csvReporter) iter(_, _ io.Writer, _ int, row []string) error { } else { _ = p.mu.csvWriter.Write(row) } - // nolint:deferunlock p.mu.Unlock() return nil } @@ -91,7 +88,6 @@ func (p *csvReporter) doneNoRows(_ io.Writer) error { return n func (p *csvReporter) doneRows(w io.Writer, seenRows int) error { p.mu.Lock() p.mu.csvWriter.Flush() - // nolint:deferunlock p.mu.Unlock() return nil } diff --git a/pkg/cli/clisqlshell/sql.go b/pkg/cli/clisqlshell/sql.go index afeb3ab774f6..6f1401918bbe 100644 --- a/pkg/cli/clisqlshell/sql.go +++ b/pkg/cli/clisqlshell/sql.go @@ -2575,7 +2575,6 @@ func (c *cliState) maybeHandleInterrupt() func() { case <-intCh: c.iCtx.mu.Lock() cancelFn, doneCh := c.iCtx.mu.cancelFn, c.iCtx.mu.doneCh - // nolint:deferunlock c.iCtx.mu.Unlock() if cancelFn == nil { // No query currently executing. @@ -2642,14 +2641,12 @@ func (c *cliState) runWithInterruptableCtx(fn func(ctx context.Context) error) e c.iCtx.mu.Lock() c.iCtx.mu.cancelFn = c.conn.Cancel c.iCtx.mu.doneCh = doneCh - // nolint:deferunlock c.iCtx.mu.Unlock() defer func() { c.iCtx.mu.Lock() cancel() c.iCtx.mu.cancelFn = nil c.iCtx.mu.doneCh = nil - // nolint:deferunlock c.iCtx.mu.Unlock() }() diff --git a/pkg/cli/syncbench/syncbench.go b/pkg/cli/syncbench/syncbench.go index 6bf4081fbca5..947703807450 100644 --- a/pkg/cli/syncbench/syncbench.go +++ b/pkg/cli/syncbench/syncbench.go @@ -109,7 +109,6 @@ func (w *worker) run(wg *sync.WaitGroup) { if err := w.latency.Current.RecordValue(elapsed.Nanoseconds()); err != nil { log.Fatalf(ctx, "%v", err) } - // nolint:deferunlock w.latency.Unlock() } } @@ -189,7 +188,6 @@ func Run(opts Options) error { w.latency.Lock() m := w.latency.Merge() w.latency.Rotate() - // nolint:deferunlock w.latency.Unlock() if h == nil { h = m diff --git a/pkg/cmd/allocsim/main.go b/pkg/cmd/allocsim/main.go index 3ff0e08de129..9882fcfa70b8 100644 --- a/pkg/cmd/allocsim/main.go +++ b/pkg/cmd/allocsim/main.go @@ -289,7 +289,6 @@ func (a *allocSim) rangeStats(d time.Duration) { stats := a.rangeInfo() a.ranges.Lock() a.ranges.stats = stats - // nolint:deferunlock a.ranges.Unlock() time.Sleep(d) @@ -343,7 +342,6 @@ func (a *allocSim) monitor(d time.Duration) { a.ranges.Lock() rangeStats := a.ranges.stats - // nolint:deferunlock a.ranges.Unlock() if ticks%20 == 0 || numReplicas != len(rangeStats.replicas) { diff --git a/pkg/cmd/roachtest/cluster.go b/pkg/cmd/roachtest/cluster.go index 1dce57980c04..151d19e8824e 100644 --- a/pkg/cmd/roachtest/cluster.go +++ b/pkg/cmd/roachtest/cluster.go @@ -501,7 +501,6 @@ func (r *clusterRegistry) countForTag(tag string) int { func (r *clusterRegistry) markClusterAsSaved(c *clusterImpl, msg string) { r.mu.Lock() r.mu.savedClusters[c] = msg - // nolint:deferunlock r.mu.Unlock() } @@ -549,7 +548,6 @@ func (r *clusterRegistry) destroyAllClusters(ctx context.Context, l *logger.Logg for c := range r.mu.savedClusters { savedClusters[c] = struct{}{} } - // nolint:deferunlock r.mu.Unlock() var wg sync.WaitGroup @@ -1107,7 +1105,6 @@ func (c *clusterImpl) Save(ctx context.Context, msg string, l *logger.Logger) { c.destroyState.mu.Lock() c.destroyState.mu.saved = true c.destroyState.mu.savedMsg = msg - // nolint:deferunlock c.destroyState.mu.Unlock() } @@ -1667,7 +1664,6 @@ func (c *clusterImpl) doDestroy(ctx context.Context, l *logger.Logger) <-chan st c.destroyState.mu.Lock() if c.destroyState.mu.saved { // Nothing to do. Short-circuit. - // nolint:deferunlock c.destroyState.mu.Unlock() ch := make(chan struct{}) close(ch) @@ -1722,7 +1718,6 @@ func (c *clusterImpl) doDestroy(ctx context.Context, l *logger.Logger) <-chan st c.destroyState.mu.Lock() ch := c.destroyState.mu.destroyed close(ch) - // nolint:deferunlock c.destroyState.mu.Unlock() return ch } diff --git a/pkg/cmd/roachtest/test_impl.go b/pkg/cmd/roachtest/test_impl.go index 4833365f8382..3e5a070b9ffa 100644 --- a/pkg/cmd/roachtest/test_impl.go +++ b/pkg/cmd/roachtest/test_impl.go @@ -528,7 +528,6 @@ func (w *workerStatus) Status() string { func (w *workerStatus) SetStatus(status string) { w.mu.Lock() w.mu.status = status - // nolint:deferunlock w.mu.Unlock() } @@ -541,7 +540,6 @@ func (w *workerStatus) Cluster() *clusterImpl { func (w *workerStatus) SetCluster(c *clusterImpl) { w.mu.Lock() w.mu.c = c - // nolint:deferunlock w.mu.Unlock() } @@ -561,7 +559,6 @@ func (w *workerStatus) SetTest(t *testImpl, ttr testToRunRes) { w.mu.Lock() w.mu.t = t w.mu.ttr = ttr - // nolint:deferunlock w.mu.Unlock() } diff --git a/pkg/cmd/roachtest/test_runner.go b/pkg/cmd/roachtest/test_runner.go index 14d8cc8a1914..d700f86ad560 100644 --- a/pkg/cmd/roachtest/test_runner.go +++ b/pkg/cmd/roachtest/test_runner.go @@ -941,7 +941,6 @@ func (r *testRunner) runTest( r.status.Lock() r.status.running[t] = struct{}{} - // nolint:deferunlock r.status.Unlock() t.runner = callerName() @@ -965,7 +964,6 @@ func (r *testRunner) runTest( t.mu.Lock() t.mu.done = true - // nolint:deferunlock t.mu.Unlock() if s.Skip != "" { @@ -1078,7 +1076,6 @@ func (r *testRunner) runTest( t.mu.Lock() // t.Fatal() will cancel this context. t.mu.cancel = cancel - // nolint:deferunlock t.mu.Unlock() // We run the actual test in a different goroutine because it might call @@ -1466,7 +1463,6 @@ func (r *testRunner) addWorker(ctx context.Context, name string) *workerStatus { func (r *testRunner) removeWorker(ctx context.Context, name string) { r.workersMu.Lock() delete(r.workersMu.workers, name) - // nolint:deferunlock r.workersMu.Unlock() } @@ -1521,7 +1517,6 @@ func (r *testRunner) serveHTTP(wr http.ResponseWriter, req *http.Request) { workers[i] = w i++ } - // nolint:deferunlock r.workersMu.Unlock() sort.Slice(workers, func(i int, j int) bool { l := workers[i] @@ -1611,7 +1606,6 @@ func (r *testRunner) serveHTTP(wr http.ResponseWriter, req *http.Request) { func (r *testRunner) recordTestFinish(info completedTestInfo) { r.completedTestsMu.Lock() r.completedTestsMu.completed = append(r.completedTestsMu.completed, info) - // nolint:deferunlock r.completedTestsMu.Unlock() } diff --git a/pkg/cmd/zerosum/main.go b/pkg/cmd/zerosum/main.go index 45e0074c9a89..fd09c1f5e38c 100644 --- a/pkg/cmd/zerosum/main.go +++ b/pkg/cmd/zerosum/main.go @@ -206,7 +206,6 @@ func (z *zeroSum) worker() { z.accounts.Lock() z.accounts.m[from] = struct{}{} z.accounts.m[to] = struct{}{} - // nolint:deferunlock z.accounts.Unlock() } } @@ -360,7 +359,6 @@ func (z *zeroSum) rangeStats(d time.Duration) { count, replicas := z.rangeInfo() z.ranges.Lock() z.ranges.count, z.ranges.replicas = count, replicas - // nolint:deferunlock z.ranges.Unlock() time.Sleep(d) @@ -399,7 +397,6 @@ func (z *zeroSum) monitor(d time.Duration) { z.ranges.Lock() ranges, replicas := z.ranges.count, z.ranges.replicas - // nolint:deferunlock z.ranges.Unlock() fmt.Printf("%8s %9d %11d %8.1f %8d %8d %8d %8d %20s\n", diff --git a/pkg/config/system.go b/pkg/config/system.go index 6ffe06cbb933..91bea3f4cb6c 100644 --- a/pkg/config/system.go +++ b/pkg/config/system.go @@ -141,7 +141,6 @@ func (s *SystemConfig) getSystemTenantDesc(key roachpb.Key) *roachpb.Value { testingLock.Lock() _, ok := testingZoneConfig[ObjectID(id)] - // nolint:deferunlock testingLock.Unlock() if ok { @@ -206,7 +205,6 @@ func (s *SystemConfig) GetLargestObjectID( ) (ObjectID, error) { testingLock.Lock() hook := testingLargestIDHook - // nolint:deferunlock testingLock.Unlock() if hook != nil { return hook(maxReservedDescID), nil @@ -428,14 +426,12 @@ func (s *SystemConfig) PurgeZoneConfigCache() { func (s *SystemConfig) getZoneEntry(codec keys.SQLCodec, id ObjectID) (zoneEntry, error) { s.mu.RLock() entry, ok := s.mu.zoneCache[id] - // nolint:deferunlock s.mu.RUnlock() if ok { return entry, nil } testingLock.Lock() hook := ZoneConfigHook - // nolint:deferunlock testingLock.Unlock() zone, placeholder, cache, err := hook(s, codec, id) if err != nil { @@ -455,7 +451,6 @@ func (s *SystemConfig) getZoneEntry(codec keys.SQLCodec, id ObjectID) (zoneEntry if cache { s.mu.Lock() s.mu.zoneCache[id] = entry - // nolint:deferunlock s.mu.Unlock() } return entry, nil @@ -724,7 +719,6 @@ func (s *SystemConfig) shouldSplitOnSystemTenantObject(id ObjectID) bool { { s.mu.RLock() shouldSplit, ok := s.mu.shouldSplitCache[id] - // nolint:deferunlock s.mu.RUnlock() if ok { return shouldSplit @@ -746,7 +740,6 @@ func (s *SystemConfig) shouldSplitOnSystemTenantObject(id ObjectID) bool { // Populate the cache. s.mu.Lock() s.mu.shouldSplitCache[id] = shouldSplit - // nolint:deferunlock s.mu.Unlock() return shouldSplit } diff --git a/pkg/gossip/client.go b/pkg/gossip/client.go index 239800d9acfa..173ac2cdc473 100644 --- a/pkg/gossip/client.go +++ b/pkg/gossip/client.go @@ -130,7 +130,6 @@ func (c *client) startLocked( } else { log.Infof(ctx, "closing client to %s: %s", c.addr, err) } - // nolint:deferunlock g.mu.RUnlock() } } @@ -159,7 +158,6 @@ func (c *client) requestGossip(g *Gossip, stream Gossip_GossipClient) error { HighWaterStamps: g.mu.is.getHighWaterStamps(), ClusterID: g.clusterID.Get(), } - // nolint:deferunlock g.mu.RUnlock() bytesSent := int64(args.Size()) diff --git a/pkg/gossip/client_test.go b/pkg/gossip/client_test.go index fd8303299fb5..0c01ef7208db 100644 --- a/pkg/gossip/client_test.go +++ b/pkg/gossip/client_test.go @@ -163,7 +163,6 @@ func gossipSucceedsSoon( g := gossip[client] g.mu.Lock() client.startLocked(g, disconnected, rpcContext, stopper) - // nolint:deferunlock g.mu.Unlock() default: } @@ -321,7 +320,6 @@ func TestClientNodeID(t *testing.T) { // The client hasn't been started or failed to start, loop and try again. local.mu.Lock() c.startLocked(local, disconnected, rpcContext, stopper) - // nolint:deferunlock local.mu.Unlock() } } @@ -345,7 +343,6 @@ func TestClientDisconnectLoopback(t *testing.T) { local.mu.Lock() lAddr := local.mu.is.NodeAddr local.startClientLocked(lAddr, localCtx) - // nolint:deferunlock local.mu.Unlock() local.manage(localCtx) testutils.SucceedsSoon(t, func() error { @@ -375,9 +372,7 @@ func TestClientDisconnectRedundant(t *testing.T) { remote.mu.Lock() rAddr := remote.mu.is.NodeAddr lAddr := local.mu.is.NodeAddr - // nolint:deferunlock local.mu.Unlock() - // nolint:deferunlock remote.mu.Unlock() local.manage(localCtx) remote.manage(remoteCtx) @@ -394,7 +389,6 @@ func TestClientDisconnectRedundant(t *testing.T) { // a heartbeat time. local.mu.Lock() local.startClientLocked(rAddr, localCtx) - // nolint:deferunlock local.mu.Unlock() return fmt.Errorf("unable to find local to remote client") } @@ -406,7 +400,6 @@ func TestClientDisconnectRedundant(t *testing.T) { // redundant as there is already a connection between the two nodes. remote.mu.Lock() remote.startClientLocked(lAddr, remoteCtx) - // nolint:deferunlock remote.mu.Unlock() testutils.SucceedsSoon(t, func() error { @@ -443,9 +436,7 @@ func TestClientDisallowMultipleConns(t *testing.T) { // connections. local.startClientLocked(rAddr, localCtx) local.startClientLocked(rAddr, localCtx) - // nolint:deferunlock local.mu.Unlock() - // nolint:deferunlock remote.mu.Unlock() local.manage(localCtx) remote.manage(remoteCtx) @@ -457,9 +448,7 @@ func TestClientDisallowMultipleConns(t *testing.T) { remote.mu.Lock() outgoing := local.outgoing.len() incoming := remote.mu.incoming.len() - // nolint:deferunlock local.mu.Unlock() - // nolint:deferunlock remote.mu.Unlock() if outgoing == 1 && incoming == 1 && verifyServerMaps(local, 0) && verifyServerMaps(remote, 1) { return nil @@ -542,7 +531,6 @@ func TestClientForwardUnresolved(t *testing.T) { } local.mu.Lock() local.outgoing.addPlaceholder() // so that the resolvePlaceholder in handleResponse doesn't fail - // nolint:deferunlock local.mu.Unlock() if err := client.handleResponse( context.Background(), local, reply, diff --git a/pkg/gossip/gossip.go b/pkg/gossip/gossip.go index 641ff948e825..480410606ecb 100644 --- a/pkg/gossip/gossip.go +++ b/pkg/gossip/gossip.go @@ -311,7 +311,6 @@ func New( // Add ourselves as a node descriptor watcher. g.mu.is.registerCallback(MakePrefixPattern(KeyNodeDescPrefix), g.updateNodeAddress) g.mu.is.registerCallback(MakePrefixPattern(KeyStoreDescPrefix), g.updateStoreMap) - // nolint:deferunlock g.mu.Unlock() return g @@ -552,7 +551,6 @@ func (g *Gossip) LogStatus() { if g.mu.is.getInfo(KeySentinel) == nil { status = redact.SafeString("stalled") } - // nolint:deferunlock g.mu.RUnlock() ctx := g.AnnotateCtx(context.TODO()) @@ -947,7 +945,6 @@ func (g *Gossip) GetClusterID() (uuid.UUID, error) { func (g *Gossip) GetInfo(key string) ([]byte, error) { g.mu.RLock() i := g.mu.is.getInfo(key) - // nolint:deferunlock g.mu.RUnlock() if i != nil { @@ -1012,7 +1009,6 @@ func (g *Gossip) tryClearInfoWithTTL(key string, ttl time.Duration) (bool, error func (g *Gossip) InfoOriginatedHere(key string) bool { g.mu.RLock() info := g.mu.is.getInfo(key) - // nolint:deferunlock g.mu.RUnlock() return info != nil && info.NodeID == g.NodeID.Get() } @@ -1076,12 +1072,10 @@ var Redundant redundantCallbacks func (g *Gossip) RegisterCallback(pattern string, method Callback, opts ...CallbackOption) func() { g.mu.Lock() unregister := g.mu.is.registerCallback(pattern, method, opts...) - // nolint:deferunlock g.mu.Unlock() return func() { g.mu.Lock() unregister() - // nolint:deferunlock g.mu.Unlock() } } @@ -1322,7 +1316,6 @@ func (g *Gossip) manage(rpcContext *rpc.Context) { if log.V(1) { g.clientsMu.Lock() log.Dev.Infof(ctx, "couldn't find least useful client among %+v", g.clientsMu.clients) - // nolint:deferunlock g.clientsMu.Unlock() } } @@ -1335,7 +1328,6 @@ func (g *Gossip) manage(rpcContext *rpc.Context) { g.mu.Lock() g.maybeSignalStatusChangeLocked() - // nolint:deferunlock g.mu.Unlock() } } diff --git a/pkg/gossip/gossip_test.go b/pkg/gossip/gossip_test.go index 881396816ae0..4ee8fd47282d 100644 --- a/pkg/gossip/gossip_test.go +++ b/pkg/gossip/gossip_test.go @@ -130,7 +130,6 @@ func TestGossipGetNextBootstrapAddress(t *testing.T) { if addrStr := addr.String(); addrStr != expAddresses[i] { t.Errorf("%d: expected addr %s; got %s", i, expAddresses[i], addrStr) } - // nolint:deferunlock g.mu.Unlock() } } @@ -234,7 +233,6 @@ func TestGossipRaceLogStatus(t *testing.T) { local.mu.Lock() peer, _ := startGossip(clusterID, 2, stopper, t, metric.NewRegistry()) local.startClientLocked(peer.mu.is.NodeAddr, localCtx) - // nolint:deferunlock local.mu.Unlock() // Race gossiping against LogStatus. @@ -283,7 +281,6 @@ func TestGossipOutgoingLimitEnforced(t *testing.T) { local, localCtx := startGossip(clusterID, 1, stopper, t, metric.NewRegistry()) local.mu.Lock() localAddr := local.mu.is.NodeAddr - // nolint:deferunlock local.mu.Unlock() var peers []*Gossip for i := 0; i < 4; i++ { @@ -293,7 +290,6 @@ func TestGossipOutgoingLimitEnforced(t *testing.T) { newPeer, peerCtx := startGossip(clusterID, roachpb.NodeID(i+2), stopper, t, metric.NewRegistry()) newPeer.mu.Lock() newPeer.startClientLocked(localAddr, peerCtx) - // nolint:deferunlock newPeer.mu.Unlock() peers = append(peers, newPeer) } @@ -318,7 +314,6 @@ func TestGossipOutgoingLimitEnforced(t *testing.T) { copy.Value.Timestamp.WallTime++ return local.mu.is.addInfo(key, ©) }, true /* deleteExpired */) - // nolint:deferunlock local.mu.Unlock() if err != nil { t.Fatal(err) @@ -334,7 +329,6 @@ func TestGossipOutgoingLimitEnforced(t *testing.T) { if numClients := len(local.clientsMu.clients); numClients > maxPeers { t.Errorf("local gossip has %d clients; the max should be %d", numClients, maxPeers) } - // nolint:deferunlock local.clientsMu.Unlock() } @@ -347,11 +341,9 @@ func TestGossipMostDistant(t *testing.T) { connect := func(from, to *Gossip, fromCtx *rpc.Context) { to.mu.Lock() addr := to.mu.is.NodeAddr - // nolint:deferunlock to.mu.Unlock() from.mu.Lock() from.startClientLocked(addr, fromCtx) - // nolint:deferunlock from.mu.Unlock() } @@ -360,7 +352,6 @@ func TestGossipMostDistant(t *testing.T) { distantNodeID, distantHops := g.mu.is.mostDistant(func(roachpb.NodeID) bool { return false }) - // nolint:deferunlock g.mu.Unlock() return distantNodeID, distantHops } @@ -434,7 +425,6 @@ func TestGossipMostDistant(t *testing.T) { } return nil }, true /* deleteExpired */) - // nolint:deferunlock g.mu.Unlock() distantNodeID, distantHops := mostDistant(g) @@ -485,7 +475,6 @@ func TestGossipNoForwardSelf(t *testing.T) { local.server.mu.Lock() maxSize := local.server.mu.incoming.maxSize - // nolint:deferunlock local.server.mu.Unlock() for i := 0; i < maxSize; i++ { g, gCtx := startGossip(clusterID, roachpb.NodeID(i+2), stopper, t, metric.NewRegistry()) @@ -525,7 +514,6 @@ func TestGossipNoForwardSelf(t *testing.T) { for i := 0; i < numClients; i++ { local.server.mu.Lock() maxSize := local.server.mu.incoming.maxSize - // nolint:deferunlock local.server.mu.Unlock() peer, peerCtx := startGossip(clusterID, roachpb.NodeID(i+maxSize+2), stopper, t, metric.NewRegistry()) @@ -534,7 +522,6 @@ func TestGossipNoForwardSelf(t *testing.T) { c := newClient(log.MakeTestingAmbientCtxWithNewTracer(), localAddr, makeMetrics()) peer.mu.Lock() c.startLocked(peer, disconnectedCh, peerCtx, stopper) - // nolint:deferunlock peer.mu.Unlock() disconnectedClient := <-disconnectedCh @@ -573,7 +560,6 @@ func TestGossipCullNetwork(t *testing.T) { peer, peerCtx := startGossip(clusterID, roachpb.NodeID(i+2), stopper, t, metric.NewRegistry()) local.startClientLocked(*peer.GetNodeAddr(), peerCtx) } - // nolint:deferunlock local.mu.Unlock() const slowGossipDuration = time.Minute @@ -628,7 +614,6 @@ func TestGossipOrphanedStallDetection(t *testing.T) { local.mu.Lock() local.startClientLocked(*peerAddr, localCtx) - // nolint:deferunlock local.mu.Unlock() testutils.SucceedsSoon(t, func() error { @@ -776,7 +761,6 @@ func TestGossipJoinTwoClusters(t *testing.T) { if a, e := len(g[1].mu.nodeMap), 0; a != e { t.Errorf("expected %v to contain %d nodes, got %d", g[1].mu.nodeMap, e, a) } - // nolint:deferunlock g[1].mu.Unlock() } @@ -795,7 +779,6 @@ func TestGossipPropagation(t *testing.T) { remote, remoteCtx := startGossip(clusterID, 2, stopper, t, metric.NewRegistry()) remote.mu.Lock() rAddr := remote.mu.is.NodeAddr - // nolint:deferunlock remote.mu.Unlock() local.manage(localCtx) remote.manage(remoteCtx) @@ -816,7 +799,6 @@ func TestGossipPropagation(t *testing.T) { // a heartbeat timeout. local.mu.Lock() local.startClientLocked(rAddr, localCtx) - // nolint:deferunlock local.mu.Unlock() return fmt.Errorf("unable to find local to remote client") } @@ -913,7 +895,6 @@ func TestGossipLoopbackInfoPropagation(t *testing.T) { remote, remoteCtx := startGossip(clusterID, 2, stopper, t, metric.NewRegistry()) remote.mu.Lock() rAddr := remote.mu.is.NodeAddr - // nolint:deferunlock remote.mu.Unlock() local.manage(localCtx) remote.manage(remoteCtx) @@ -943,7 +924,6 @@ func TestGossipLoopbackInfoPropagation(t *testing.T) { // Start a client connection to the remote node. local.mu.Lock() local.startClientLocked(rAddr, localCtx) - // nolint:deferunlock local.mu.Unlock() getInfo := func(g *Gossip, key string) *Info { diff --git a/pkg/gossip/infostore.go b/pkg/gossip/infostore.go index 43becb0137e7..d71b7355b45e 100644 --- a/pkg/gossip/infostore.go +++ b/pkg/gossip/infostore.go @@ -109,7 +109,6 @@ func ratchetMonotonic(v int64) { if monoTime.last < v { monoTime.last = v } - // nolint:deferunlock monoTime.Unlock() } @@ -180,7 +179,6 @@ func newInfoStore( is.callbackWorkMu.Lock() work := is.callbackWork is.callbackWork = nil - // nolint:deferunlock is.callbackWorkMu.Unlock() if len(work) == 0 { @@ -344,7 +342,6 @@ func (is *infoStore) runCallbacks(key string, content roachpb.Value, callbacks . } is.callbackWorkMu.Lock() is.callbackWork = append(is.callbackWork, f) - // nolint:deferunlock is.callbackWorkMu.Unlock() // Signal the callback goroutine. Callbacks run in a goroutine to avoid mutex diff --git a/pkg/gossip/infostore_test.go b/pkg/gossip/infostore_test.go index b72e3e3d8bb9..a682cb0e9234 100644 --- a/pkg/gossip/infostore_test.go +++ b/pkg/gossip/infostore_test.go @@ -202,7 +202,6 @@ func TestCombineInfosRatchetMonotonic(t *testing.T) { // Reset the monotonic clock. monoTime.Lock() monoTime.last = 0 - // nolint:deferunlock monoTime.Unlock() fresh, err := is.combine(map[string]*Info{"hello": info}, 2) @@ -217,7 +216,6 @@ func TestCombineInfosRatchetMonotonic(t *testing.T) { // locally. monoTime.Lock() last := monoTime.last - // nolint:deferunlock monoTime.Unlock() var expectedLast int64 if local { diff --git a/pkg/gossip/server.go b/pkg/gossip/server.go index 46b809000cdb..bbdd9d00f162 100644 --- a/pkg/gossip/server.go +++ b/pkg/gossip/server.go @@ -330,7 +330,6 @@ func (s *server) gossipReceiver( cycler.Wait() } - // nolint:deferunlock s.mu.Unlock() recvArgs, err := receiverFn() s.mu.Lock() @@ -390,7 +389,6 @@ func (s *server) start(addr net.Addr) { s.mu.Lock() unregister() - // nolint:deferunlock s.mu.Unlock() broadcast() diff --git a/pkg/internal/rsg/rsg.go b/pkg/internal/rsg/rsg.go index 4fb3a3cc9ecc..6139ab0954b3 100644 --- a/pkg/internal/rsg/rsg.go +++ b/pkg/internal/rsg/rsg.go @@ -67,7 +67,6 @@ func (r *RSG) Generate(root string, depth int) string { } else { s = "" } - // nolint:deferunlock r.lock.Unlock() } if s != "" { @@ -181,7 +180,6 @@ func (r *RSG) GenerateRandomArg(typ *types.T) string { r.lock.Lock() datum := randgen.RandDatumWithNullChance(r.Rnd, typ, 0, /* nullChance */ false /* favorCommonData */, false /* targetColumnIsUnique */) - // nolint:deferunlock r.lock.Unlock() return tree.Serialize(datum) @@ -196,7 +194,6 @@ type lockedSource struct { func (r *lockedSource) Int63() (n int64) { r.lk.Lock() n = r.src.Int63() - // nolint:deferunlock r.lk.Unlock() return } @@ -204,7 +201,6 @@ func (r *lockedSource) Int63() (n int64) { func (r *lockedSource) Uint64() (n uint64) { r.lk.Lock() n = r.src.Uint64() - // nolint:deferunlock r.lk.Unlock() return } @@ -212,6 +208,5 @@ func (r *lockedSource) Uint64() (n uint64) { func (r *lockedSource) Seed(seed int64) { r.lk.Lock() r.src.Seed(seed) - // nolint:deferunlock r.lk.Unlock() } diff --git a/pkg/internal/sqlsmith/bulkio.go b/pkg/internal/sqlsmith/bulkio.go index 82b8cbd3b2ce..740b1c3126c9 100644 --- a/pkg/internal/sqlsmith/bulkio.go +++ b/pkg/internal/sqlsmith/bulkio.go @@ -104,7 +104,6 @@ func makeBackup(s *Smither) (tree.Statement, bool) { } s.lock.Lock() s.bulkBackups[name] = targets - // nolint:deferunlock s.lock.Unlock() coinD := tree.DBoolFalse @@ -129,7 +128,6 @@ func makeRestore(s *Smither) (tree.Statement, bool) { } // Only restore each backup once. delete(s.bulkBackups, name) - // nolint:deferunlock s.lock.Unlock() if name == "" { @@ -194,7 +192,6 @@ func makeExport(s *Smither) (tree.Statement, bool) { s.lock.Lock() s.bulkFiles[fmt.Sprintf("/%s%s", exp, exportSchema)] = []byte(schema) s.bulkExports = append(s.bulkExports, string(exp)) - // nolint:deferunlock s.lock.Unlock() return &tree.Export{ @@ -213,7 +210,6 @@ func makeImport(s *Smither) (tree.Statement, bool) { s.lock.Lock() if len(s.bulkExports) == 0 { - // nolint:deferunlock s.lock.Unlock() return nil, false } @@ -241,7 +237,6 @@ func makeImport(s *Smither) (tree.Statement, bool) { s.bulkFiles[schema], []byte(fmt.Sprintf("CREATE TABLE %s (", tab)), ) - // nolint:deferunlock s.lock.Unlock() // Create the table to be imported into. diff --git a/pkg/internal/sqlsmith/relational.go b/pkg/internal/sqlsmith/relational.go index 6a3174afd668..e913c2cc9e65 100644 --- a/pkg/internal/sqlsmith/relational.go +++ b/pkg/internal/sqlsmith/relational.go @@ -1104,7 +1104,6 @@ func (s *Smither) makeCreateFunc() (cf *tree.CreateRoutine, ok bool) { def: tree.NewFunctionDefinition(name.String(), &tree.FunctionProperties{}, nil /* def */), overload: ov, }) - // nolint:deferunlock functions.Unlock() return stmt, true } diff --git a/pkg/internal/sqlsmith/sampler.go b/pkg/internal/sqlsmith/sampler.go index b4e0c365476f..f1a69337fccd 100644 --- a/pkg/internal/sqlsmith/sampler.go +++ b/pkg/internal/sqlsmith/sampler.go @@ -63,7 +63,6 @@ type statementSampler struct { func (w *statementSampler) Next() statement { w.mu.Lock() v := w.samples[w.rnd.Intn(len(w.samples))] - // nolint:deferunlock w.mu.Unlock() return v } @@ -121,7 +120,6 @@ type tableExprSampler struct { func (w *tableExprSampler) Next() tableExpr { w.mu.Lock() v := w.samples[w.rnd.Intn(len(w.samples))] - // nolint:deferunlock w.mu.Unlock() return v } @@ -181,7 +179,6 @@ type selectStatementSampler struct { func (w *selectStatementSampler) Next() selectStatement { w.mu.Lock() v := w.samples[w.rnd.Intn(len(w.samples))] - // nolint:deferunlock w.mu.Unlock() return v } @@ -239,7 +236,6 @@ type scalarExprSampler struct { func (w *scalarExprSampler) Next() scalarExpr { w.mu.Lock() v := w.samples[w.rnd.Intn(len(w.samples))] - // nolint:deferunlock w.mu.Unlock() return v } diff --git a/pkg/internal/sqlsmith/scalar.go b/pkg/internal/sqlsmith/scalar.go index e8cfe80adb40..1eb551ad72d3 100644 --- a/pkg/internal/sqlsmith/scalar.go +++ b/pkg/internal/sqlsmith/scalar.go @@ -415,7 +415,6 @@ func makeFunc(s *Smither, ctx Context, typ *types.T, refs colRefs) (tree.TypedEx } functions.Lock() fns := functions.fns[class][typ.Oid()] - // nolint:deferunlock functions.Unlock() if len(fns) == 0 { return nil, false diff --git a/pkg/internal/sqlsmith/schema.go b/pkg/internal/sqlsmith/schema.go index bfffbdb8da57..f346e2e1bf95 100644 --- a/pkg/internal/sqlsmith/schema.go +++ b/pkg/internal/sqlsmith/schema.go @@ -115,7 +115,6 @@ func (s *Smither) getRandTableIndex( ) (*tree.TableIndexName, *tree.CreateIndex, colRefs, bool) { s.lock.RLock() indexes := s.indexes[table] - // nolint:deferunlock s.lock.RUnlock() if len(indexes) == 0 { return nil, nil, nil, false diff --git a/pkg/jobs/adopt.go b/pkg/jobs/adopt.go index acae25ad4004..bedbd9d7f506 100644 --- a/pkg/jobs/adopt.go +++ b/pkg/jobs/adopt.go @@ -433,7 +433,6 @@ func (r *Registry) runJob( } username := job.mu.payload.UsernameProto.Decode() typ := job.mu.payload.Type() - // nolint:deferunlock job.mu.Unlock() // Make sure that we remove the job from the running set when this returns. diff --git a/pkg/jobs/job_scheduler.go b/pkg/jobs/job_scheduler.go index 1cb783282259..f7d6523f76f1 100644 --- a/pkg/jobs/job_scheduler.go +++ b/pkg/jobs/job_scheduler.go @@ -365,7 +365,6 @@ func newCancelWhenDisabled(sv *settings.Values) *syncCancelFunc { if sf.CancelFunc != nil { sf.CancelFunc() } - // nolint:deferunlock sf.Unlock() } }) diff --git a/pkg/jobs/jobs_test.go b/pkg/jobs/jobs_test.go index 77c57840f84c..65a3cc0efaa9 100644 --- a/pkg/jobs/jobs_test.go +++ b/pkg/jobs/jobs_test.go @@ -277,12 +277,10 @@ func (rts *registryTestSuite) setUp(t *testing.T) { } rts.mu.Lock() rts.mu.a.ResumeStart = true - // nolint:deferunlock rts.mu.Unlock() defer func() { rts.mu.Lock() rts.mu.a.ResumeExit++ - // nolint:deferunlock rts.mu.Unlock() t.Log("Exiting resume") }() @@ -292,7 +290,6 @@ func (rts *registryTestSuite) setUp(t *testing.T) { case <-ctx.Done(): rts.mu.Lock() rts.mu.a.ResumeExit-- - // nolint:deferunlock rts.mu.Unlock() return ctx.Err() case err := <-rts.resumeCh: @@ -314,20 +311,17 @@ func (rts *registryTestSuite) setUp(t *testing.T) { } rts.mu.Lock() rts.mu.a.OnFailOrCancelStart = true - // nolint:deferunlock rts.mu.Unlock() <-rts.failOrCancelCheckCh select { case <-ctx.Done(): rts.mu.Lock() rts.mu.a.OnFailOrCancelExit = false - // nolint:deferunlock rts.mu.Unlock() return ctx.Err() case err := <-rts.failOrCancelCh: rts.mu.Lock() rts.mu.a.OnFailOrCancelExit = true - // nolint:deferunlock rts.mu.Unlock() t.Log("Exiting FailOrCancel") return err @@ -338,7 +332,6 @@ func (rts *registryTestSuite) setUp(t *testing.T) { t.Log("Starting success") rts.mu.Lock() defer func() { - // nolint:deferunlock rts.mu.Unlock() t.Log("Exiting success") }() @@ -3180,7 +3173,6 @@ func (r *resumeStartedSignaler) SignalResumeStarted() { r.Lock() r.isStarted = true r.cond.Signal() - // nolint:deferunlock r.Unlock() } @@ -3190,7 +3182,6 @@ func (r *resumeStartedSignaler) WaitForResumeStarted() { r.cond.Wait() } r.isStarted = false - // nolint:deferunlock r.Unlock() } diff --git a/pkg/jobs/progress.go b/pkg/jobs/progress.go index bed31c0928d0..0450f0f1eb96 100644 --- a/pkg/jobs/progress.go +++ b/pkg/jobs/progress.go @@ -150,7 +150,6 @@ func (p *ProgressUpdateBatcher) Add(ctx context.Context, delta float32) error { p.reported = p.completed p.lastReported = timeutil.Now() } - // nolint:deferunlock p.Unlock() if shouldReport { @@ -165,7 +164,6 @@ func (p *ProgressUpdateBatcher) Done(ctx context.Context) error { p.Lock() completed := p.completed shouldReport := completed-p.reported > progressFractionThreshold - // nolint:deferunlock p.Unlock() if shouldReport { diff --git a/pkg/jobs/registry.go b/pkg/jobs/registry.go index 3c904c04e351..792f5a750ea9 100644 --- a/pkg/jobs/registry.go +++ b/pkg/jobs/registry.go @@ -2044,7 +2044,6 @@ func (r *Registry) DrainRequested(ctx context.Context) { alreadyDraining := r.mu.draining numWait := r.mu.numDrainWait r.mu.draining = true - // nolint:deferunlock r.mu.Unlock() if alreadyDraining { @@ -2067,7 +2066,6 @@ func (r *Registry) DrainRequested(ctx context.Context) { case <-r.jobDrained: r.mu.Lock() numWait = r.mu.numDrainWait - // nolint:deferunlock r.mu.Unlock() } } @@ -2079,13 +2077,11 @@ func (r *Registry) DrainRequested(ctx context.Context) { func (r *Registry) OnDrain() (<-chan struct{}, func()) { r.mu.Lock() r.mu.numDrainWait++ - // nolint:deferunlock r.mu.Unlock() return r.drainRequested, func() { r.mu.Lock() r.mu.numDrainWait-- - // nolint:deferunlock r.mu.Unlock() select { diff --git a/pkg/kv/bulk/buffering_adder.go b/pkg/kv/bulk/buffering_adder.go index 8617e4c47b54..44a419d0f7d1 100644 --- a/pkg/kv/bulk/buffering_adder.go +++ b/pkg/kv/bulk/buffering_adder.go @@ -159,7 +159,6 @@ func (b *BufferingAdder) Close(ctx context.Context) { } else { log.Infof(ctx, "%s adder closing; ingested nothing", b.name) } - // nolint:deferunlock b.sink.mu.Unlock() } b.sink.Close(ctx) @@ -271,7 +270,6 @@ func (b *BufferingAdder) doFlush(ctx context.Context, forSize bool) error { before.Combine(&b.sink.mu.totalStats) before.Combine(&b.sink.currentStats) beforeSize = b.sink.mu.totalBulkOpSummary.DataSize - // nolint:deferunlock b.sink.mu.Unlock() } @@ -327,7 +325,6 @@ func (b *BufferingAdder) doFlush(ctx context.Context, forSize bool) error { afterStats := b.sink.mu.totalStats.Identity().(*bulkpb.IngestionPerformanceStats) afterStats.Combine(&b.sink.mu.totalStats) afterStats.Combine(&b.sink.currentStats) - // nolint:deferunlock b.sink.mu.Unlock() files := afterStats.Batches - before.Batches @@ -354,14 +351,12 @@ func (b *BufferingAdder) doFlush(ctx context.Context, forSize bool) error { if log.V(3) { b.sink.mu.totalStats.LogPerStoreTimings(ctx, b.name) } - // nolint:deferunlock b.sink.mu.Unlock() } if log.V(3) { b.sink.mu.Lock() b.sink.mu.totalStats.LogFlushes(ctx, b.name, "flushed", b.memAcc.Used(), b.sink.span) - // nolint:deferunlock b.sink.mu.Unlock() } diff --git a/pkg/kv/bulk/sst_batcher.go b/pkg/kv/bulk/sst_batcher.go index ff5002b851fe..97ca41e42d02 100644 --- a/pkg/kv/bulk/sst_batcher.go +++ b/pkg/kv/bulk/sst_batcher.go @@ -838,7 +838,6 @@ func (b *SSTBatcher) addSSTable( if b.writeAtBatchTS { b.mu.maxWriteTS.Forward(br.Timestamp) } - // nolint:deferunlock b.mu.Unlock() // If this was sent async then, by the time the reply gets back, it // might not be the last range anymore. We can just discard the last diff --git a/pkg/kv/kvclient/kvcoord/dist_sender_mux_rangefeed.go b/pkg/kv/kvclient/kvcoord/dist_sender_mux_rangefeed.go index 038f33eb8fd1..4c0e2105bd59 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender_mux_rangefeed.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender_mux_rangefeed.go @@ -594,7 +594,6 @@ func (c *muxStream) startRangeFeed( func (c *muxStream) lookupStream(streamID int64) (a *activeMuxRangeFeed) { c.mu.Lock() a = c.mu.streams[streamID] - // nolint:deferunlock c.mu.Unlock() return a } @@ -602,7 +601,6 @@ func (c *muxStream) lookupStream(streamID int64) (a *activeMuxRangeFeed) { func (c *muxStream) deleteStream(streamID int64) { c.mu.Lock() delete(c.mu.streams, streamID) - // nolint:deferunlock c.mu.Unlock() } @@ -615,7 +613,6 @@ func (c *muxStream) close() []*activeMuxRangeFeed { toRestart = append(toRestart, a) } c.mu.streams = nil - // nolint:deferunlock c.mu.Unlock() return toRestart diff --git a/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed_test.go b/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed_test.go index d7930bbbce46..2452db21890b 100644 --- a/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed_test.go +++ b/pkg/kv/kvclient/kvcoord/dist_sender_rangefeed_test.go @@ -752,7 +752,6 @@ func TestRangeFeedMetricsManagement(t *testing.T) { // Return transient error for this range, but do this only once per range. skipSet.Lock() skipSet.retry.Add(checkpoint.Span) - // nolint:deferunlock skipSet.Unlock() log.Infof(ctx, "skipping span %s", checkpoint.Span) *event = transientErrEvent @@ -770,7 +769,6 @@ func TestRangeFeedMetricsManagement(t *testing.T) { // and arrange for other events belonging to this range to be skipped as well. skipSet.Lock() skipSet.stuck.Add(checkpoint.Span) - // nolint:deferunlock skipSet.Unlock() log.Infof(ctx, "skipping stuck span %s", checkpoint.Span) return true /* skip */, nil diff --git a/pkg/kv/kvclient/kvcoord/txn_correctness_test.go b/pkg/kv/kvclient/kvcoord/txn_correctness_test.go index 27ccf6abe296..a92497295e77 100644 --- a/pkg/kv/kvclient/kvcoord/txn_correctness_test.go +++ b/pkg/kv/kvclient/kvcoord/txn_correctness_test.go @@ -926,7 +926,6 @@ func (hv *historyVerifier) runCmd( cmdStr := fmt.Sprintf(fmtStr, txnIdx+1, retry) hv.mu.Lock() hv.mu.actual = append(hv.mu.actual, cmdStr) - // nolint:deferunlock hv.mu.Unlock() return cmdStr, err } diff --git a/pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater.go b/pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater.go index 2a21e4865405..38de47c9bf53 100644 --- a/pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater.go +++ b/pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater.go @@ -364,7 +364,6 @@ func (h *txnHeartbeater) heartbeatLoop(ctx context.Context) { defer func() { h.mu.Lock() h.cancelHeartbeatLoopLocked() - // nolint:deferunlock h.mu.Unlock() }() diff --git a/pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater_test.go b/pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater_test.go index d23c427ece0f..880167be07a9 100644 --- a/pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater_test.go +++ b/pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater_test.go @@ -169,7 +169,6 @@ func TestTxnHeartbeaterLoopStartedOnFirstLock(t *testing.T) { th.mu.Lock() require.False(t, th.mu.loopStarted) require.False(t, th.heartbeatLoopRunningLocked()) - // nolint:deferunlock th.mu.Unlock() // The heartbeat loop is started on the first locking request. @@ -187,7 +186,6 @@ func TestTxnHeartbeaterLoopStartedOnFirstLock(t *testing.T) { th.mu.Lock() require.True(t, th.mu.loopStarted) require.True(t, th.heartbeatLoopRunningLocked()) - // nolint:deferunlock th.mu.Unlock() // The interceptor indicates whether the heartbeat loop is @@ -211,7 +209,6 @@ func TestTxnHeartbeaterLoopStartedOnFirstLock(t *testing.T) { // Closing the interceptor stops the heartbeat loop. th.mu.Lock() th.closeLocked() - // nolint:deferunlock th.mu.Unlock() waitForHeartbeatLoopToStop(t, &th) require.True(t, th.mu.loopStarted) // still set @@ -314,7 +311,6 @@ func TestTxnHeartbeaterLoopStartsBeforeExpiry(t *testing.T) { th.mu.Lock() require.False(t, th.mu.loopStarted) require.False(t, th.heartbeatLoopRunningLocked()) - // nolint:deferunlock th.mu.Unlock() count := 0 @@ -364,7 +360,6 @@ func TestTxnHeartbeaterLoopStartsBeforeExpiry(t *testing.T) { // push delay). require.Positivef(t, count, "expected heartbeat before starting loop") } - // nolint:deferunlock th.mu.Unlock() if test.loopStarts == StartBeforeInterval { @@ -433,7 +428,6 @@ func TestTxnHeartbeaterLoopStartedFor1PC(t *testing.T) { require.True(t, th.mu.loopStarted) require.True(t, th.heartbeatLoopRunningLocked()) th.closeLocked() - // nolint:deferunlock th.mu.Unlock() } @@ -763,7 +757,6 @@ func TestTxnHeartbeaterAsyncAbortCollapsesRequests(t *testing.T) { th.mu.Lock() // manually lock, there's no TxnCoordSender br, pErr := th.SendLocked(ctx, ba) - // nolint:deferunlock th.mu.Unlock() require.Nil(t, pErr) require.NotNil(t, br) @@ -786,7 +779,6 @@ func TestTxnHeartbeaterAsyncAbortCollapsesRequests(t *testing.T) { th.mu.Lock() // manually lock, there's no TxnCoordSender br, pErr = th.SendLocked(ctx, ba) - // nolint:deferunlock th.mu.Unlock() require.Nil(t, pErr) require.NotNil(t, br) @@ -839,7 +831,6 @@ func TestTxnHeartbeaterEndTxnLoopHandling(t *testing.T) { th.mu.Lock() br, pErr = th.SendLocked(ctx, ba2) - // nolint:deferunlock th.mu.Unlock() require.Nil(t, pErr) diff --git a/pkg/kv/kvclient/kvstreamer/requests_provider_test.go b/pkg/kv/kvclient/kvstreamer/requests_provider_test.go index f7a2c48e71e1..9b8b183d9c73 100644 --- a/pkg/kv/kvclient/kvstreamer/requests_provider_test.go +++ b/pkg/kv/kvclient/kvstreamer/requests_provider_test.go @@ -46,7 +46,6 @@ func TestInOrderRequestsProvider(t *testing.T) { p.Lock() next := p.nextLocked() p.removeNextLocked() - // nolint:deferunlock p.Unlock() require.Equal(t, priorities[0], next.priority()) priorities = priorities[1:] diff --git a/pkg/kv/kvclient/kvstreamer/results_buffer_test.go b/pkg/kv/kvclient/kvstreamer/results_buffer_test.go index a2d30281dcb6..32e09e1013a1 100644 --- a/pkg/kv/kvclient/kvstreamer/results_buffer_test.go +++ b/pkg/kv/kvclient/kvstreamer/results_buffer_test.go @@ -122,9 +122,7 @@ func TestInOrderResultsBuffer(t *testing.T) { addOrder = addOrder[1:] } b.doneAddingLocked(ctx) - // nolint:deferunlock b.Unlock() - // nolint:deferunlock budget.mu.Unlock() // With 50% probability, try spilling some of the buffered results diff --git a/pkg/kv/kvclient/kvstreamer/streamer.go b/pkg/kv/kvclient/kvstreamer/streamer.go index 593f64499f25..6ba5527be620 100644 --- a/pkg/kv/kvclient/kvstreamer/streamer.go +++ b/pkg/kv/kvclient/kvstreamer/streamer.go @@ -766,7 +766,6 @@ func (s *Streamer) Close(ctx context.Context) { s.coordinatorCtxCancel() s.mu.Lock() s.mu.done = true - // nolint:deferunlock s.mu.Unlock() s.requestsToServe.close() // Unblock the coordinator in case it is waiting for the budget. @@ -857,7 +856,6 @@ func (w *workerCoordinator) mainLoop(ctx context.Context) { // with less urgency when necessary to free up the budget. spillingPriority = w.s.requestsToServe.nextLocked().priority() } - // nolint:deferunlock w.s.requestsToServe.Unlock() avgResponseSize, shouldExit := w.getAvgResponseSize() @@ -927,7 +925,6 @@ func (w *workerCoordinator) waitForRequests(ctx context.Context) error { } w.s.mu.Lock() shouldExit := w.s.results.error() != nil || w.s.mu.done - // nolint:deferunlock w.s.mu.Unlock() if shouldExit { return nil diff --git a/pkg/kv/kvclient/kvtenant/connector.go b/pkg/kv/kvclient/kvtenant/connector.go index e639674ac2b4..88efb4d82bd9 100644 --- a/pkg/kv/kvclient/kvtenant/connector.go +++ b/pkg/kv/kvclient/kvtenant/connector.go @@ -910,7 +910,6 @@ func (c *connector) getClient(ctx context.Context) (*client, error) { ctx = c.AnnotateCtx(ctx) c.mu.RLock() if client := c.mu.client; client != nil { - // nolint:deferunlock c.mu.RUnlock() return client, nil } @@ -936,7 +935,6 @@ func (c *connector) getClient(ctx context.Context) (*client, error) { c.mu.client = client return client, nil }) - // nolint:deferunlock c.mu.RUnlock() res := future.WaitForResult(ctx) diff --git a/pkg/kv/kvclient/rangecache/range_cache.go b/pkg/kv/kvclient/rangecache/range_cache.go index f43dfb629c6b..63fd541ff0dc 100644 --- a/pkg/kv/kvclient/rangecache/range_cache.go +++ b/pkg/kv/kvclient/rangecache/range_cache.go @@ -763,7 +763,6 @@ func (rc *RangeCache) tryLookup( ) (EvictionToken, error) { rc.rangeCache.RLock() if entry, _ := rc.getCachedRLocked(ctx, key, useReverseScan); entry != nil { - // nolint:deferunlock rc.rangeCache.RUnlock() returnToken := rc.makeEvictionToken(entry, nil /* nextDesc */) return returnToken, nil diff --git a/pkg/kv/kvclient/rangecache/range_cache_test.go b/pkg/kv/kvclient/rangecache/range_cache_test.go index da55ca7e40aa..950434d37dc7 100644 --- a/pkg/kv/kvclient/rangecache/range_cache_test.go +++ b/pkg/kv/kvclient/rangecache/range_cache_test.go @@ -1346,7 +1346,6 @@ func TestGetCachedRangeDescriptorInverted(t *testing.T) { t.Run("", func(t *testing.T) { cache.rangeCache.RLock() targetRange, _ := cache.getCachedRLocked(ctx, test.queryKey, true /* inverted */) - // nolint:deferunlock cache.rangeCache.RUnlock() if test.rng == nil { diff --git a/pkg/kv/kvclient/rangefeed/rangefeed_external_test.go b/pkg/kv/kvclient/rangefeed/rangefeed_external_test.go index b779342330da..f01f4bfdf65e 100644 --- a/pkg/kv/kvclient/rangefeed/rangefeed_external_test.go +++ b/pkg/kv/kvclient/rangefeed/rangefeed_external_test.go @@ -256,7 +256,6 @@ func TestWithOnFrontierAdvance(t *testing.T) { require.NoError(t, db.Put(ctx, mkKey("c"), 1)) mu.Lock() secondWriteFinished = true - // nolint:deferunlock mu.Unlock() v = <-rows @@ -323,7 +322,6 @@ func TestWithOnCheckpoint(t *testing.T) { case c := <-checkpoints: mu.RLock() writeTSUnset := afterWriteTS.IsEmpty() - // nolint:deferunlock mu.RUnlock() if writeTSUnset { return errors.New("write to key hasn't gone through yet") @@ -360,7 +358,6 @@ func TestWithOnCheckpoint(t *testing.T) { require.NoError(t, db.Put(ctx, mkKey("a"), 1)) mu.Lock() afterWriteTS = db.Clock().Now() - // nolint:deferunlock mu.Unlock() { v := <-rows diff --git a/pkg/kv/kvclient/rangefeed/rangefeedcache/watcher.go b/pkg/kv/kvclient/rangefeed/rangefeedcache/watcher.go index 7953592c386e..ae1c492e1f66 100644 --- a/pkg/kv/kvclient/rangefeed/rangefeedcache/watcher.go +++ b/pkg/kv/kvclient/rangefeed/rangefeedcache/watcher.go @@ -258,7 +258,6 @@ func (s *Watcher) Run(ctx context.Context) error { defer func() { mu.Lock() s.lastFrontierTS.Forward(mu.frontierTS) - // nolint:deferunlock mu.Unlock() }() @@ -298,7 +297,6 @@ func (s *Watcher) Run(ctx context.Context) error { rangefeed.WithOnFrontierAdvance(func(ctx context.Context, frontierTS hlc.Timestamp) { mu.Lock() mu.frontierTS = frontierTS - // nolint:deferunlock mu.Unlock() select { @@ -348,7 +346,6 @@ func (s *Watcher) Run(ctx context.Context) error { case <-frontierBumpedCh: mu.Lock() frontierTS := mu.frontierTS - // nolint:deferunlock mu.Unlock() s.handleUpdate(ctx, buffer, frontierTS, IncrementalUpdate) diff --git a/pkg/kv/kvnemesis/applier.go b/pkg/kv/kvnemesis/applier.go index 1bf42bdb211c..0b85f3dd78b6 100644 --- a/pkg/kv/kvnemesis/applier.go +++ b/pkg/kv/kvnemesis/applier.go @@ -75,7 +75,6 @@ func (a *Applier) getNextDBRoundRobin() (*kv.DB, int32) { a.mu.Lock() dbIdx := a.mu.dbIdx a.mu.dbIdx = (a.mu.dbIdx + 1) % len(a.dbs) - // nolint:deferunlock a.mu.Unlock() return a.dbs[dbIdx], int32(dbIdx) } diff --git a/pkg/kv/kvnemesis/watcher.go b/pkg/kv/kvnemesis/watcher.go index 84c2560864d1..c3ac5d532d4d 100644 --- a/pkg/kv/kvnemesis/watcher.go +++ b/pkg/kv/kvnemesis/watcher.go @@ -84,7 +84,6 @@ func Watch(ctx context.Context, env *Env, dbs []*kv.DB, dataSpan roachpb.Span) ( for i := 0; ; i = (i + 1) % len(dbs) { w.mu.Lock() ts.Forward(w.mu.frontier.Frontier()) - // nolint:deferunlock w.mu.Unlock() ds := dss[i] @@ -147,7 +146,6 @@ func (w *Watcher) WaitForFrontier(ctx context.Context, ts hlc.Timestamp) (retErr resultCh := make(chan error, 1) w.mu.Lock() w.mu.frontierWaiters[ts] = append(w.mu.frontierWaiters[ts], resultCh) - // nolint:deferunlock w.mu.Unlock() select { case <-ctx.Done(): diff --git a/pkg/kv/kvserver/allocator/allocatorimpl/allocator_scorer.go b/pkg/kv/kvserver/allocator/allocatorimpl/allocator_scorer.go index fd6195c0fbfe..ed2997788cd7 100644 --- a/pkg/kv/kvserver/allocator/allocatorimpl/allocator_scorer.go +++ b/pkg/kv/kvserver/allocator/allocatorimpl/allocator_scorer.go @@ -988,7 +988,6 @@ func (cl candidateList) selectBest(randGen allocatorRand) *candidate { } randGen.Lock() order := randGen.Perm(len(cl)) - // nolint:deferunlock randGen.Unlock() best := &cl[order[0]] for i := 1; i < allocatorRandomCount; i++ { @@ -1011,7 +1010,6 @@ func (cl candidateList) selectGood(randGen allocatorRand) *candidate { } randGen.Lock() r := randGen.Intn(len(cl)) - // nolint:deferunlock randGen.Unlock() c := &cl[r] return c @@ -1029,7 +1027,6 @@ func (cl candidateList) selectWorst(randGen allocatorRand) *candidate { } randGen.Lock() order := randGen.Perm(len(cl)) - // nolint:deferunlock randGen.Unlock() worst := &cl[order[0]] for i := 1; i < allocatorRandomCount; i++ { diff --git a/pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go b/pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go index 9c5e3bb2baf2..33e39bfcd89d 100644 --- a/pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go +++ b/pkg/kv/kvserver/allocator/allocatorimpl/allocator_test.go @@ -1358,7 +1358,6 @@ func TestAllocatorRebalanceDeadNodes(t *testing.T) { sp.GetStoreDetailLocked(6).Desc.Capacity = ranges(0) sp.GetStoreDetailLocked(7).Desc.Capacity = ranges(100) sp.GetStoreDetailLocked(8).Desc.Capacity = ranges(100) - // nolint:deferunlock sp.DetailsMu.Unlock() // Each test case should describe a repair situation which has a lower diff --git a/pkg/kv/kvserver/allocator/storepool/override_store_pool_test.go b/pkg/kv/kvserver/allocator/storepool/override_store_pool_test.go index 8cfd670947b7..9fd8d7396264 100644 --- a/pkg/kv/kvserver/allocator/storepool/override_store_pool_test.go +++ b/pkg/kv/kvserver/allocator/storepool/override_store_pool_test.go @@ -336,7 +336,6 @@ func TestOverrideStorePoolGetStoreList(t *testing.T) { // Set suspectedStore as suspected. testStorePool.DetailsMu.Lock() testStorePool.DetailsMu.StoreDetails[suspectedStore.StoreID].LastUnavailable = testStorePool.clock.Now() - // nolint:deferunlock testStorePool.DetailsMu.Unlock() // No filter or limited set of store IDs. diff --git a/pkg/kv/kvserver/allocator/storepool/store_pool.go b/pkg/kv/kvserver/allocator/storepool/store_pool.go index 42ab8d066fd4..66c9357d6f66 100644 --- a/pkg/kv/kvserver/allocator/storepool/store_pool.go +++ b/pkg/kv/kvserver/allocator/storepool/store_pool.go @@ -454,13 +454,11 @@ func (sp *StorePool) storeDescriptorUpdate(storeDesc roachpb.StoreDescriptor) { } detail.Desc = &storeDesc detail.LastUpdatedTime = now - // nolint:deferunlock sp.DetailsMu.Unlock() sp.localitiesMu.Lock() sp.localitiesMu.nodeLocalities[storeDesc.Node.NodeID] = localityWithString{storeDesc.Node.Locality, storeDesc.Node.Locality.String()} - // nolint:deferunlock sp.localitiesMu.Unlock() if oldCapacity != curCapacity { diff --git a/pkg/kv/kvserver/allocator/storepool/store_pool_test.go b/pkg/kv/kvserver/allocator/storepool/store_pool_test.go index dbd678bcebe5..080a86ab0d90 100644 --- a/pkg/kv/kvserver/allocator/storepool/store_pool_test.go +++ b/pkg/kv/kvserver/allocator/storepool/store_pool_test.go @@ -65,7 +65,6 @@ func TestStorePoolGossipUpdate(t *testing.T) { if _, ok := sp.DetailsMu.StoreDetails[2]; ok { t.Fatalf("store 2 is already in the pool's store list") } - // nolint:deferunlock sp.DetailsMu.RUnlock() sg.GossipStores(uniqueStore, t) @@ -74,7 +73,6 @@ func TestStorePoolGossipUpdate(t *testing.T) { if _, ok := sp.DetailsMu.StoreDetails[2]; !ok { t.Fatalf("store 2 isn't in the pool's store list") } - // nolint:deferunlock sp.DetailsMu.RUnlock() } @@ -211,7 +209,6 @@ func TestStorePoolGetStoreList(t *testing.T) { sp.DetailsMu.StoreDetails[declinedStore.StoreID].ThrottledUntil = sp.clock.Now().AddDuration(time.Hour) // Set suspectedStore as suspected. sp.DetailsMu.StoreDetails[suspectedStore.StoreID].LastUnavailable = sp.clock.Now() - // nolint:deferunlock sp.DetailsMu.Unlock() // No filter or limited set of store IDs. @@ -597,7 +594,6 @@ func TestStorePoolThrottle(t *testing.T) { sp.DetailsMu.Lock() detail := sp.GetStoreDetailLocked(1) - // nolint:deferunlock sp.DetailsMu.Unlock() if detail.ThrottledUntil.WallTime != expected.WallTime { t.Errorf("expected store to have been throttled to %v, found %v", diff --git a/pkg/kv/kvserver/allocator_impl_test.go b/pkg/kv/kvserver/allocator_impl_test.go index c5f02044912c..15a1c318bb9e 100644 --- a/pkg/kv/kvserver/allocator_impl_test.go +++ b/pkg/kv/kvserver/allocator_impl_test.go @@ -301,7 +301,6 @@ func TestAllocatorRebalanceTarget(t *testing.T) { repl.mu.Lock() repl.mu.state.Stats = &enginepb.MVCCStats{} - // nolint:deferunlock repl.mu.Unlock() repl.loadStats = load.NewReplicaLoad(clock, nil) @@ -421,7 +420,6 @@ func TestAllocatorThrottled(t *testing.T) { t.Fatalf("store:%d was not found in the store pool", singleStore[0].StoreID) } storeDetail.ThrottledUntil = hlc.Timestamp{WallTime: timeutil.Now().Add(24 * time.Hour).UnixNano()} - // nolint:deferunlock sp.DetailsMu.Unlock() _, _, err = a.AllocateVoter(ctx, sp, simpleSpanConfig, []roachpb.ReplicaDescriptor{}, nil, nil, allocatorimpl.Dead) if _, ok := IsPurgatoryError(err); ok { diff --git a/pkg/kv/kvserver/client_merge_test.go b/pkg/kv/kvserver/client_merge_test.go index e7ca467e84b8..dc5f76fe1db6 100644 --- a/pkg/kv/kvserver/client_merge_test.go +++ b/pkg/kv/kvserver/client_merge_test.go @@ -451,7 +451,6 @@ func mergeCheckingTimestampCaches( filterMu.Lock() mergeCommitFilterCopy := filterMu.mergeCommitFilter blockHBAndGCsCopy := filterMu.blockHBAndGCs - // nolint:deferunlock filterMu.Unlock() for _, req := range ba.Requests { switch v := req.GetInner().(type) { @@ -479,7 +478,6 @@ func mergeCheckingTimestampCaches( ) error { filterMu.Lock() snapshotFilterCopy := snapshotFilter - // nolint:deferunlock filterMu.Unlock() if snapshotFilterCopy != nil { snapshotFilterCopy(inSnap) @@ -817,7 +815,6 @@ func mergeCheckingTimestampCaches( close(filterMu.blockHBAndGCs) filterMu.Lock() filterMu.blockHBAndGCs = nil - // nolint:deferunlock filterMu.Unlock() t.Logf("waiting for snapshot to LHS leaseholder") @@ -3231,7 +3228,6 @@ func (h *slowSnapRaftHandler) unblock() { close(h.waitCh) h.waitCh = nil } - // nolint:deferunlock h.Unlock() } @@ -3243,7 +3239,6 @@ func (h *slowSnapRaftHandler) HandleSnapshot( if header.RaftMessageRequest.RangeID == h.rangeID { h.Lock() waitCh := h.waitCh - // nolint:deferunlock h.Unlock() if waitCh != nil { <-waitCh @@ -4137,7 +4132,6 @@ func TestStoreRangeMergeDuringShutdown(t *testing.T) { // task from launching. This error path would previously fatal a node // incorrectly (#27552). state.stopping = true - // nolint:deferunlock state.Unlock() go s.Stopper().Stop(ctx) // Sleep to give the shutdown time to propagate. The test appeared to work @@ -4173,7 +4167,6 @@ func TestStoreRangeMergeDuringShutdown(t *testing.T) { rhsDesc := store.LookupReplica(roachpb.RKey(key)).Desc() state.Lock() state.rhsDesc = rhsDesc - // nolint:deferunlock state.Unlock() // Simulate a merge transaction by launching a transaction that lays down @@ -4190,7 +4183,6 @@ func TestStoreRangeMergeDuringShutdown(t *testing.T) { // acquisition for the RHS should trigger a shutdown. state.Lock() state.stop = true - // nolint:deferunlock state.Unlock() // Expire all leases. diff --git a/pkg/kv/kvserver/client_replica_test.go b/pkg/kv/kvserver/client_replica_test.go index 5cdee74871ce..4ba9302d755c 100644 --- a/pkg/kv/kvserver/client_replica_test.go +++ b/pkg/kv/kvserver/client_replica_test.go @@ -1407,7 +1407,6 @@ func setupLeaseTransferTest(t *testing.T) *leaseTransferTest { testingEvalFilter := func(args kvserverbase.FilterArgs) *kvpb.Error { l.filterMu.Lock() filterCopy := l.evalFilter - // nolint:deferunlock l.filterMu.Unlock() if filterCopy != nil { return filterCopy(args) @@ -1418,7 +1417,6 @@ func setupLeaseTransferTest(t *testing.T) *leaseTransferTest { testingProposalFilter := func(args kvserverbase.ProposalFilterArgs) *kvpb.Error { l.filterMu.Lock() filterCopy := l.propFilter - // nolint:deferunlock l.filterMu.Unlock() if filterCopy != nil { return filterCopy(args) @@ -1536,7 +1534,6 @@ func (l *leaseTransferTest) setFilter(setTo bool, extensionSem chan struct{}) { // the signal to proceed. l.filterMu.Lock() l.evalFilter = nil - // nolint:deferunlock l.filterMu.Unlock() extensionSem <- struct{}{} log.Infof(filterArgs.Ctx, "filter blocking request: %s", llReq) @@ -4323,7 +4320,6 @@ func TestStrictGCEnforcement(t *testing.T) { isBlocked.Set(true) <-blockKVSubscriberCh } - // nolint:deferunlock mu.Unlock() // Ensure that the KVSubscriber has been blocked. @@ -4369,7 +4365,6 @@ func TestStrictGCEnforcement(t *testing.T) { isBlocked.Set(true) <-blockKVSubscriberCh } - // nolint:deferunlock mu.Unlock() // Ensure that the KVSubscriber has been blocked. diff --git a/pkg/kv/kvserver/closedts/sidetransport/debug.go b/pkg/kv/kvserver/closedts/sidetransport/debug.go index 06f9a7ebe0cf..3f69cfd5cf44 100644 --- a/pkg/kv/kvserver/closedts/sidetransport/debug.go +++ b/pkg/kv/kvserver/closedts/sidetransport/debug.go @@ -36,7 +36,6 @@ func (s *Receiver) HTML() string { for _, c := range s.mu.conns { conns = append(conns, c) } - // nolint:deferunlock s.mu.RUnlock() // Sort by node id. sort.Slice(conns, func(i, j int) bool { @@ -52,7 +51,6 @@ func (s *Receiver) HTML() string { for _, c := range s.historyMu.lastClosed { closed = append(closed, c) } - // nolint:deferunlock s.historyMu.Unlock() // Sort by disconnection time, descending. sort.Slice(closed, func(i, j int) bool { @@ -106,7 +104,6 @@ func (s *Sender) HTML() string { header("Closed timestamps sender state") s.leaseholdersMu.Lock() fmt.Fprintf(sb, "leaseholders: %d\n", len(s.leaseholdersMu.leaseholders)) - // nolint:deferunlock s.leaseholdersMu.Unlock() s.trackedMu.Lock() diff --git a/pkg/kv/kvserver/closedts/sidetransport/receiver.go b/pkg/kv/kvserver/closedts/sidetransport/receiver.go index 1b860f44c6bc..9ffb35137136 100644 --- a/pkg/kv/kvserver/closedts/sidetransport/receiver.go +++ b/pkg/kv/kvserver/closedts/sidetransport/receiver.go @@ -102,7 +102,6 @@ func (s *Receiver) GetClosedTimestamp( ) (hlc.Timestamp, kvpb.LeaseAppliedIndex) { s.mu.RLock() conn, ok := s.mu.conns[leaseholderNode] - // nolint:deferunlock s.mu.RUnlock() if !ok { return hlc.Timestamp{}, 0 @@ -158,7 +157,6 @@ func (s *Receiver) onRecvErr(ctx context.Context, nodeID roachpb.NodeID, err err closeErr: err, closeTime: timeutil.Now(), } - // nolint:deferunlock s.historyMu.Unlock() } } @@ -255,7 +253,6 @@ func (r *incomingStream) processUpdate(ctx context.Context, msg *ctpb.Update) { r.stores.ForwardSideTransportClosedTimestampForRange( ctx, rangeID, r.mu.lastClosed[info.policy], info.lai) } - // nolint:deferunlock r.mu.RUnlock() } diff --git a/pkg/kv/kvserver/closedts/sidetransport/sender.go b/pkg/kv/kvserver/closedts/sidetransport/sender.go index 9ff1c4be49d1..2652947ed180 100644 --- a/pkg/kv/kvserver/closedts/sidetransport/sender.go +++ b/pkg/kv/kvserver/closedts/sidetransport/sender.go @@ -357,7 +357,6 @@ func (s *Sender) publish(ctx context.Context) hlc.ClockTimestamp { for k, v := range s.leaseholdersMu.leaseholders { leaseholders[k] = v } - // nolint:deferunlock s.leaseholdersMu.Unlock() // We'll accumulate all the nodes we need to connect to in order to check if @@ -747,7 +746,6 @@ func (r *rpcConn) cleanupStream(err error) { r.mu.state.connected = false r.mu.state.lastDisconnect = err r.mu.state.lastDisconnectTime = timeutil.Now() - // nolint:deferunlock r.mu.Unlock() } @@ -871,7 +869,6 @@ func (r *rpcConn) recordConnect() { r.mu.Lock() r.mu.state.connected = true r.mu.state.connectedTime = timeutil.Now() - // nolint:deferunlock r.mu.Unlock() } diff --git a/pkg/kv/kvserver/closedts/sidetransport/sender_test.go b/pkg/kv/kvserver/closedts/sidetransport/sender_test.go index 493b419644b2..e44afd7ed6eb 100644 --- a/pkg/kv/kvserver/closedts/sidetransport/sender_test.go +++ b/pkg/kv/kvserver/closedts/sidetransport/sender_test.go @@ -334,7 +334,6 @@ var _ ctpb.SideTransportServer = &mockReceiver{} func (s *mockReceiver) PushUpdates(stream ctpb.SideTransport_PushUpdatesServer) error { s.mu.Lock() s.mu.called = true - // nolint:deferunlock s.mu.Unlock() // Block the RPC until close() is called. <-s.stop diff --git a/pkg/kv/kvserver/closedts/tracker/tracker_test.go b/pkg/kv/kvserver/closedts/tracker/tracker_test.go index 11319668a68c..54719ce695b9 100644 --- a/pkg/kv/kvserver/closedts/tracker/tracker_test.go +++ b/pkg/kv/kvserver/closedts/tracker/tracker_test.go @@ -448,7 +448,6 @@ func (c *trackerChecker) run(ctx context.Context) error { minEvalTS := int64(math.MaxInt64) for _, req := range c.mu.requests.all() { if req.wtsNanos < lbNanos { - // nolint:deferunlock c.mu.Unlock() return fmt.Errorf("bad lower bound %d > req: %d", lbNanos, req.wtsNanos) } @@ -550,7 +549,6 @@ func benchmarkTracker(ctx context.Context, b *testing.B, t Tracker) { mu.RLock() tok := t.Track(ctx, i) toks[myid] = append(toks[myid], tok) - // nolint:deferunlock mu.RUnlock() } }) diff --git a/pkg/kv/kvserver/concurrency/concurrency_manager_test.go b/pkg/kv/kvserver/concurrency/concurrency_manager_test.go index d3b60f6536e8..07591fff9eff 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_manager_test.go +++ b/pkg/kv/kvserver/concurrency/concurrency_manager_test.go @@ -248,7 +248,6 @@ func TestConcurrencyManagerBasic(t *testing.T) { c.mu.Lock() prev := c.guardsByReqName[reqName] delete(c.guardsByReqName, reqName) - // nolint:deferunlock c.mu.Unlock() opName := fmt.Sprintf("sequence %s", reqName) @@ -262,7 +261,6 @@ func TestConcurrencyManagerBasic(t *testing.T) { log.Event(ctx, "sequencing complete, returned guard") c.mu.Lock() c.guardsByReqName[reqName] = guard - // nolint:deferunlock c.mu.Unlock() } else { log.Event(ctx, "sequencing complete, returned no guard") @@ -284,7 +282,6 @@ func TestConcurrencyManagerBasic(t *testing.T) { m.FinishReq(guard) c.mu.Lock() delete(c.guardsByReqName, reqName) - // nolint:deferunlock c.mu.Unlock() }) return c.waitAndCollect(t, mon) @@ -350,13 +347,11 @@ func TestConcurrencyManagerBasic(t *testing.T) { log.Eventf(ctx, "handled %v, returned error: %v", lcErr, err) c.mu.Lock() delete(c.guardsByReqName, reqName) - // nolint:deferunlock c.mu.Unlock() } else { log.Eventf(ctx, "handled %v, released latches", lcErr) c.mu.Lock() c.guardsByReqName[reqName] = guard - // nolint:deferunlock c.mu.Unlock() } }) @@ -1016,7 +1011,6 @@ func (c *cluster) reset() error { r.mu.Lock() r.updatedStatus = roachpb.PENDING r.updatedTimestamp = hlc.Timestamp{} - // nolint:deferunlock r.mu.Unlock() } // There should be no remaining concurrency guards. diff --git a/pkg/kv/kvserver/concurrency/lock_table.go b/pkg/kv/kvserver/concurrency/lock_table.go index 56f48f07ec3f..9d1049328c18 100644 --- a/pkg/kv/kvserver/concurrency/lock_table.go +++ b/pkg/kv/kvserver/concurrency/lock_table.go @@ -576,7 +576,6 @@ func (g *lockTableGuardImpl) CurState() waitingState { // Not actively waiting anywhere so no one else can set // mustComputeWaitingState to true while this method executes. g.mu.mustComputeWaitingState = false - // nolint:deferunlock g.mu.Unlock() g.resumeScan(false /* notify */) g.mu.Lock() // Unlock deferred @@ -1831,7 +1830,6 @@ func (kl *keyLocks) lockStateInfo(now time.Time) roachpb.LockStateInfo { Strength: lock.None, WaitDuration: now.Sub(readerGuard.mu.curLockWaitStart), }) - // nolint:deferunlock readerGuard.mu.Unlock() } @@ -1951,7 +1949,6 @@ func (kl *keyLocks) informActiveWaiters() { // If the state has indeed changed, it must perform a different action -- so // we pass notify = true here to nudge it to do so. g.maybeUpdateWaitingStateLocked(state, true /* notify */) - // nolint:deferunlock g.mu.Unlock() } for e := kl.queuedLockingRequests.Front(); e != nil; e = e.Next() { @@ -1981,7 +1978,6 @@ func (kl *keyLocks) informActiveWaiters() { // If the state has indeed changed, it must perform a different action -- so // we pass notify = true here to nudge it to do so. g.maybeUpdateWaitingStateLocked(state, true /* notify */) - // nolint:deferunlock g.mu.Unlock() } } @@ -2076,7 +2072,6 @@ func (kl *keyLocks) tryMakeNewDistinguished() { g.mu.state.kind = waitForDistinguished // The rest of g.state is already up-to-date. g.notify() - // nolint:deferunlock g.mu.Unlock() } } @@ -2173,7 +2168,6 @@ func (kl *keyLocks) totalAndMaxWaitDuration(now time.Time) (time.Duration, time. if waitDuration > maxWaitDuration { maxWaitDuration = waitDuration } - // nolint:deferunlock g.mu.Unlock() } for e := kl.queuedLockingRequests.Front(); e != nil; e = e.Next() { @@ -2185,7 +2179,6 @@ func (kl *keyLocks) totalAndMaxWaitDuration(now time.Time) (time.Duration, time. if waitDuration > maxWaitDuration { maxWaitDuration = waitDuration } - // nolint:deferunlock g.mu.Unlock() } return totalWaitDuration, maxWaitDuration @@ -2940,7 +2933,6 @@ func (kl *keyLocks) discoveredLock( // Since g will place itself in queue as inactive waiter below. g.mu.locks[kl] = struct{}{} } - // nolint:deferunlock g.mu.Unlock() if !presentHere { @@ -3234,7 +3226,6 @@ func (kl *keyLocks) removeReader(e *list.Element[*lockTableGuardImpl]) bool { g.mu.Lock() delete(g.mu.locks, kl) g.doneActivelyWaitingAtLock() - // nolint:deferunlock g.mu.Unlock() if g == kl.distinguishedWaiter { kl.distinguishedWaiter = nil @@ -3523,7 +3514,6 @@ func (t *lockTableImpl) ScanAndEnqueue(req Request, guard lockTableGuard) lockTa g.mu.startWait = false g.mu.state = waitingState{} g.mu.mustComputeWaitingState = false - // nolint:deferunlock g.mu.Unlock() g.toResolve = g.toResolve[:0] } @@ -3570,7 +3560,6 @@ func (t *lockTableImpl) doSnapshotForGuard(g *lockTableGuardImpl) { t.locks.mu.RLock() g.tableSnapshot.Reset() g.tableSnapshot = t.locks.Clone() - // nolint:deferunlock t.locks.mu.RUnlock() } @@ -3591,7 +3580,6 @@ func (t *lockTableImpl) Dequeue(guard lockTableGuard) { for l := range g.mu.locks { candidateLocks = append(candidateLocks, l) } - // nolint:deferunlock g.mu.Unlock() var locksToGC []*keyLocks for _, l := range candidateLocks { @@ -3876,7 +3864,6 @@ func (t *lockTableImpl) tryGCLocks(tree *treeMu, locks []*keyLocks) { l = iter.Cur() l.mu.Lock() empty := l.isEmptyLock() - // nolint:deferunlock l.mu.Unlock() if empty { tree.Delete(l) @@ -3919,7 +3906,6 @@ func (t *lockTableImpl) updateLockInternal(up *roachpb.LockUpdate) (heldByTxn bo break } } - // nolint:deferunlock t.locks.mu.RUnlock() t.tryGCLocks(&t.locks, locksToGC) @@ -3968,7 +3954,6 @@ func (t *lockTableImpl) Enable(seq roachpb.LeaseSequence) { // NOTE: This may be a premature optimization, but it can't hurt. t.enabledMu.RLock() enabled, enabledSeq := t.enabled, t.enabledSeq - // nolint:deferunlock t.enabledMu.RUnlock() if enabled && enabledSeq == seq { return @@ -3976,7 +3961,6 @@ func (t *lockTableImpl) Enable(seq roachpb.LeaseSequence) { t.enabledMu.Lock() t.enabled = true t.enabledSeq = seq - // nolint:deferunlock t.enabledMu.Unlock() } @@ -4011,7 +3995,6 @@ func (t *lockTableImpl) QueryLockTableState( // Grab tree snapshot to avoid holding read lock during iteration. t.locks.mu.RLock() snap := t.locks.Clone() - // nolint:deferunlock t.locks.mu.RUnlock() // Reset snapshot to free resources. defer snap.Reset() @@ -4068,7 +4051,6 @@ func (t *lockTableImpl) Metrics() LockTableMetrics { // Grab tree snapshot to avoid holding read lock during iteration. t.locks.mu.RLock() snap := t.locks.Clone() - // nolint:deferunlock t.locks.mu.RUnlock() // Reset snapshot to free resources. defer snap.Reset() @@ -4092,10 +4074,8 @@ func (t *lockTableImpl) String() string { l := iter.Cur() l.mu.Lock() l.safeFormat(&sb, &t.txnStatusCache) - // nolint:deferunlock l.mu.Unlock() } - // nolint:deferunlock t.locks.mu.RUnlock() return sb.String() } diff --git a/pkg/kv/kvserver/concurrency/lock_table_test.go b/pkg/kv/kvserver/concurrency/lock_table_test.go index 73e123181bde..e0793d82ce1a 100644 --- a/pkg/kv/kvserver/concurrency/lock_table_test.go +++ b/pkg/kv/kvserver/concurrency/lock_table_test.go @@ -1391,7 +1391,6 @@ L: } e.mu.Lock() e.transactions[txnID] = tstate - // nolint:deferunlock e.mu.Unlock() } e.transactions[txnID].ongoingRequests[&work] = struct{}{} diff --git a/pkg/kv/kvserver/concurrency/lock_table_waiter.go b/pkg/kv/kvserver/concurrency/lock_table_waiter.go index 0417924c5cd4..0d189f2bf598 100644 --- a/pkg/kv/kvserver/concurrency/lock_table_waiter.go +++ b/pkg/kv/kvserver/concurrency/lock_table_waiter.go @@ -1071,13 +1071,11 @@ func newContentionEventTracer(sp *tracing.Span, clock *hlc.Clock) *contentionEve oldContentionTag.mu.Lock() waiting := oldContentionTag.mu.waiting if waiting { - // nolint:deferunlock oldContentionTag.mu.Unlock() panic("span already contains contention tag in the waiting state") } t.tag.mu.numLocks = oldContentionTag.mu.numLocks t.tag.mu.lockWait = oldContentionTag.mu.lockWait - // nolint:deferunlock oldContentionTag.mu.Unlock() } diff --git a/pkg/kv/kvserver/helpers_test.go b/pkg/kv/kvserver/helpers_test.go index c02001efa2e4..39d7c870e0ea 100644 --- a/pkg/kv/kvserver/helpers_test.go +++ b/pkg/kv/kvserver/helpers_test.go @@ -315,7 +315,6 @@ func (r *Replica) RaftLock() { } func (r *Replica) RaftUnlock() { - // nolint:deferunlock r.raftMu.Unlock() } diff --git a/pkg/kv/kvserver/idalloc/id_alloc_test.go b/pkg/kv/kvserver/idalloc/id_alloc_test.go index ac2e654f8a4b..d370ab69e977 100644 --- a/pkg/kv/kvserver/idalloc/id_alloc_test.go +++ b/pkg/kv/kvserver/idalloc/id_alloc_test.go @@ -162,7 +162,6 @@ func TestAllocateErrorAndRecovery(t *testing.T) { // Make Allocator invalid. mu.Lock() mu.err = errors.New("boom") - // nolint:deferunlock mu.Unlock() // Should be able to get the allocated IDs, and there will be one @@ -211,7 +210,6 @@ func TestAllocateErrorAndRecovery(t *testing.T) { // Make the IDAllocator valid again. mu.Lock() mu.err = nil - // nolint:deferunlock mu.Unlock() // Check if the blocked allocations return expected ID. @@ -290,7 +288,6 @@ func TestLostWriteAssertion(t *testing.T) { // Mess with the counter. mu.Lock() mu.counter-- - // nolint:deferunlock mu.Unlock() for i := 0; ; i++ { @@ -298,7 +295,6 @@ func TestLostWriteAssertion(t *testing.T) { if err != nil { mu.Lock() msg := mu.fatal - // nolint:deferunlock mu.Unlock() require.Contains(t, msg, "counter corrupt") break diff --git a/pkg/kv/kvserver/intentresolver/intent_resolver.go b/pkg/kv/kvserver/intentresolver/intent_resolver.go index becc8c6d7a71..25dd13cc061c 100644 --- a/pkg/kv/kvserver/intentresolver/intent_resolver.go +++ b/pkg/kv/kvserver/intentresolver/intent_resolver.go @@ -393,7 +393,6 @@ func (ir *IntentResolver) MaybePushTransactions( delete(ir.mu.inFlightPushes, txnID) } } - // nolint:deferunlock ir.mu.Unlock() } ir.mu.Unlock() @@ -642,7 +641,6 @@ func (ir *IntentResolver) lockInFlightTxnCleanup( return true, func() { ir.mu.Lock() delete(ir.mu.inFlightTxnCleanups, txnID) - // nolint:deferunlock ir.mu.Unlock() } } diff --git a/pkg/kv/kvserver/intentresolver/intent_resolver_test.go b/pkg/kv/kvserver/intentresolver/intent_resolver_test.go index fe48a6231778..38c747519bba 100644 --- a/pkg/kv/kvserver/intentresolver/intent_resolver_test.go +++ b/pkg/kv/kvserver/intentresolver/intent_resolver_test.go @@ -375,7 +375,6 @@ func TestCleanupMultipleIntentsAsync(t *testing.T) { for _, ru := range ba.Requests { reqs.Lock() reqs.pushed = append(reqs.pushed, string(ru.GetPushTxn().Key)) - // nolint:deferunlock reqs.Unlock() } return pushTxnSendFunc(t, len(ba.Requests))(ctx, ba) @@ -383,7 +382,6 @@ func TestCleanupMultipleIntentsAsync(t *testing.T) { for _, ru := range ba.Requests { reqs.Lock() reqs.resolved = append(reqs.resolved, string(ru.GetResolveIntent().Key)) - // nolint:deferunlock reqs.Unlock() } return resolveIntentsSendFunc(t)(ctx, ba) @@ -642,7 +640,6 @@ func TestCleanupMultipleTxnIntentsAsync(t *testing.T) { case kvpb.ResolveIntent: reqs.Lock() reqs.resolved = append(reqs.resolved, string(ru.GetResolveIntent().Key)) - // nolint:deferunlock reqs.Unlock() return resolveIntentsSendFunc(t)(ctx, ba) case kvpb.ResolveIntentRange: @@ -650,13 +647,11 @@ func TestCleanupMultipleTxnIntentsAsync(t *testing.T) { req := ru.GetResolveIntentRange() reqs.resolved = append(reqs.resolved, fmt.Sprintf("%s-%s", string(req.Key), string(req.EndKey))) - // nolint:deferunlock reqs.Unlock() return resolveIntentsSendFunc(t)(ctx, ba) case kvpb.GC: reqs.Lock() reqs.gced = append(reqs.gced, string(ru.GetGc().Key)) - // nolint:deferunlock reqs.Unlock() return gcSendFunc(t)(ctx, ba) default: @@ -879,7 +874,6 @@ func newIntentResolverWithSendFuncsConcurrentSend( sf.mu.Lock() f := sf.popLocked() if allowConcurrentSend { - // nolint:deferunlock sf.mu.Unlock() } else { defer sf.mu.Unlock() diff --git a/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle.go b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle.go index 06e5c8a42210..40efbcbf0d6a 100644 --- a/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle.go +++ b/pkg/kv/kvserver/kvflowcontrol/kvflowhandle/kvflowhandle.go @@ -99,7 +99,6 @@ func (h *Handle) Admit( h.mu.Lock() if h.mu.closed { - // nolint:deferunlock h.mu.Unlock() log.Errorf(ctx, "operating on a closed handle") return false, nil @@ -403,7 +402,6 @@ func (h *Handle) TestingNonBlockingAdmit( log.Fatalf(ctx, "operating on a closed handle") } connections := h.mu.connections - // nolint:deferunlock h.mu.Unlock() type testingNonBlockingController interface { diff --git a/pkg/kv/kvserver/liveness/cache.go b/pkg/kv/kvserver/liveness/cache.go index 88483c883de6..ef68c1203810 100644 --- a/pkg/kv/kvserver/liveness/cache.go +++ b/pkg/kv/kvserver/liveness/cache.go @@ -129,7 +129,6 @@ func (c *cache) storeGossipUpdate(_ string, content roachpb.Value) { } previousRec.lastUpdateTime = c.clock.Now() c.mu.lastNodeUpdate[nodeID] = previousRec - // nolint:deferunlock c.mu.Unlock() } diff --git a/pkg/kv/kvserver/liveness/client_test.go b/pkg/kv/kvserver/liveness/client_test.go index 11fa816d9bc7..e3fb65c5c0e0 100644 --- a/pkg/kv/kvserver/liveness/client_test.go +++ b/pkg/kv/kvserver/liveness/client_test.go @@ -271,7 +271,6 @@ func TestNodeLivenessDecommissionedCallback(t *testing.T) { cb.m = map[roachpb.NodeID]bool{} } cb.m[id] = true - // nolint:deferunlock cb.Unlock() }, }, @@ -305,7 +304,6 @@ func TestNodeLivenessDecommissionedCallback(t *testing.T) { testutils.SucceedsSoon(t, func() error { cb.Lock() sl := pretty.Diff(map[roachpb.NodeID]bool{2: true}, cb.m) - // nolint:deferunlock cb.Unlock() if len(sl) > 0 { return errors.Errorf("diff(exp,act) = %s", strings.Join(sl, "\n")) diff --git a/pkg/kv/kvserver/liveness/liveness.go b/pkg/kv/kvserver/liveness/liveness.go index a2efa0749633..e8069fa19a2f 100644 --- a/pkg/kv/kvserver/liveness/liveness.go +++ b/pkg/kv/kvserver/liveness/liveness.go @@ -1085,7 +1085,6 @@ func (nl *NodeLiveness) Metrics() Metrics { func (nl *NodeLiveness) RegisterCallback(cb IsLiveCallback) { nl.onIsLiveMu.Lock() nl.onIsLiveMu.callbacks = append(nl.onIsLiveMu.callbacks, cb) - // nolint:deferunlock nl.onIsLiveMu.Unlock() nl.notifyIsAliveCallbacks([]IsLiveCallback{cb}) diff --git a/pkg/kv/kvserver/metrics.go b/pkg/kv/kvserver/metrics.go index f0a82a344653..98ca367f6ac4 100644 --- a/pkg/kv/kvserver/metrics.go +++ b/pkg/kv/kvserver/metrics.go @@ -2698,7 +2698,6 @@ func (sm *TenantsStorageMetrics) releaseTenant(ctx context.Context, ref *tenantM } ref._stack.Lock() ref._stack.string = string(debug.Stack()) - // nolint:deferunlock ref._stack.Unlock() m.mu.Lock() defer m.mu.Unlock() @@ -3346,7 +3345,6 @@ func (sm *StoreMetrics) updateEngineMetrics(m storage.Metrics) { sm.l0SublevelsTracker.Lock() sm.l0SublevelsTracker.swag.Record(timeutil.Now(), float64(m.Levels[0].Sublevels)) curMax, _ := sm.l0SublevelsTracker.swag.Query(timeutil.Now()) - // nolint:deferunlock sm.l0SublevelsTracker.Unlock() syncutil.StoreFloat64(&sm.l0SublevelsWindowedMax, curMax) diff --git a/pkg/kv/kvserver/mvcc_gc_queue.go b/pkg/kv/kvserver/mvcc_gc_queue.go index f0d27efd1b24..cc191b10e63c 100644 --- a/pkg/kv/kvserver/mvcc_gc_queue.go +++ b/pkg/kv/kvserver/mvcc_gc_queue.go @@ -272,7 +272,6 @@ func makeMVCCGCQueueScore( ) mvccGCQueueScore { repl.mu.Lock() ms := *repl.mu.state.Stats - // nolint:deferunlock repl.mu.Unlock() if repl.store.cfg.TestingKnobs.DisableLastProcessedCheck { diff --git a/pkg/kv/kvserver/mvcc_gc_queue_test.go b/pkg/kv/kvserver/mvcc_gc_queue_test.go index a7b71fc45c21..bbcb07eaefe5 100644 --- a/pkg/kv/kvserver/mvcc_gc_queue_test.go +++ b/pkg/kv/kvserver/mvcc_gc_queue_test.go @@ -1229,9 +1229,7 @@ func TestMVCCGCQueueTransactionTable(t *testing.T) { tc.repl.raftMu.Lock() tc.repl.mu.RLock() tc.repl.assertStateRaftMuLockedReplicaMuRLocked(ctx, batch) // check that in-mem and on-disk state were updated - // nolint:deferunlock tc.repl.mu.RUnlock() - // nolint:deferunlock tc.repl.raftMu.Unlock() } diff --git a/pkg/kv/kvserver/protectedts/ptcache/cache.go b/pkg/kv/kvserver/protectedts/ptcache/cache.go index bc608610ae7d..230a625488a6 100644 --- a/pkg/kv/kvserver/protectedts/ptcache/cache.go +++ b/pkg/kv/kvserver/protectedts/ptcache/cache.go @@ -81,7 +81,6 @@ func (c *Cache) Iterate( ) (asOf hlc.Timestamp) { c.mu.RLock() state, lastUpdate := c.mu.state, c.mu.lastUpdate - // nolint:deferunlock c.mu.RUnlock() sp := roachpb.Span{ diff --git a/pkg/kv/kvserver/protectedts/ptreconcile/reconciler_test.go b/pkg/kv/kvserver/protectedts/ptreconcile/reconciler_test.go index b3f1300619ee..da2c8675c77d 100644 --- a/pkg/kv/kvserver/protectedts/ptreconcile/reconciler_test.go +++ b/pkg/kv/kvserver/protectedts/ptreconcile/reconciler_test.go @@ -104,7 +104,6 @@ func TestReconciler(t *testing.T) { t.Run("reconcile", func(t *testing.T) { state.mu.Lock() state.toRemove[recMeta] = struct{}{} - // nolint:deferunlock state.mu.Unlock() ptreconcile.ReconcileInterval.Override(ctx, &settings.SV, time.Millisecond) diff --git a/pkg/kv/kvserver/queue.go b/pkg/kv/kvserver/queue.go index 3eedbc38e3fb..c3ba4a1fe672 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -530,7 +530,6 @@ func (bq *baseQueue) PurgatoryLength() int { func (bq *baseQueue) SetDisabled(disabled bool) { bq.mu.Lock() bq.mu.disabled = disabled - // nolint:deferunlock bq.mu.Unlock() } @@ -657,7 +656,6 @@ func (bq *baseQueue) maybeAdd(ctx context.Context, repl replicaInQueue, now hlc. bq.mu.Lock() stopped := bq.mu.stopped || bq.mu.disabled - // nolint:deferunlock bq.mu.Unlock() if stopped { @@ -837,7 +835,6 @@ func (bq *baseQueue) processLoop(stopper *stop.Stopper) { done := func() { bq.mu.Lock() bq.mu.stopped = true - // nolint:deferunlock bq.mu.Unlock() } if err := stopper.RunAsyncTaskEx(ctx, @@ -1100,7 +1097,6 @@ func (bq *baseQueue) finishProcessingReplica( item.callbacks = nil bq.removeFromReplicaSetLocked(repl.GetRangeID()) item = nil // prevent accidental use below - // nolint:deferunlock bq.mu.Unlock() if !processing { @@ -1129,7 +1125,6 @@ func (bq *baseQueue) finishProcessingReplica( if purgErr, ok := IsPurgatoryError(err); ok { bq.mu.Lock() bq.addToPurgatoryLocked(ctx, stopper, repl, purgErr) - // nolint:deferunlock bq.mu.Unlock() return } @@ -1212,7 +1207,6 @@ func (bq *baseQueue) addToPurgatoryLocked( for _, err := range bq.mu.purgatory { errMap[err.Error()]++ } - // nolint:deferunlock bq.mu.Unlock() for errStr, count := range errMap { log.Errorf(ctx, "%d replicas failing with %q", count, errStr) @@ -1271,11 +1265,9 @@ func (bq *baseQueue) processReplicasInPurgatory( if len(bq.mu.purgatory) == 0 { log.Infof(ctx, "purgatory is now empty") bq.mu.purgatory = nil - // nolint:deferunlock bq.mu.Unlock() return true /* purgatoryCleared */ } - // nolint:deferunlock bq.mu.Unlock() return false /* purgatoryCleared */ } @@ -1288,7 +1280,6 @@ func (bq *baseQueue) pop() (replicaInQueue, float64) { bq.mu.Lock() for { if bq.mu.priorityQ.Len() == 0 { - // nolint:deferunlock bq.mu.Unlock() return nil, 0 } diff --git a/pkg/kv/kvserver/queue_concurrency_test.go b/pkg/kv/kvserver/queue_concurrency_test.go index 7b16c7f5a3f0..f32762ebe7aa 100644 --- a/pkg/kv/kvserver/queue_concurrency_test.go +++ b/pkg/kv/kvserver/queue_concurrency_test.go @@ -128,7 +128,6 @@ func TestBaseQueueConcurrent(t *testing.T) { for done := false; !done; { bq.mu.Lock() done = len(bq.mu.replicas) == 0 - // nolint:deferunlock bq.mu.Unlock() runtime.Gosched() } diff --git a/pkg/kv/kvserver/queue_test.go b/pkg/kv/kvserver/queue_test.go index 254d5b9dba16..de7fe8137b02 100644 --- a/pkg/kv/kvserver/queue_test.go +++ b/pkg/kv/kvserver/queue_test.go @@ -1306,7 +1306,6 @@ func TestQueueDisable(t *testing.T) { } testCase.queue.mu.Lock() disabled := testCase.queue.mu.disabled - // nolint:deferunlock testCase.queue.mu.Unlock() if disabled != true { t.Errorf("%s should be disabled", testCase.name) @@ -1430,7 +1429,6 @@ func TestBaseQueueChangeReplicaID(t *testing.T) { } return r, nil } - // nolint:deferunlock bq.mu.Unlock() require.Equal(t, 0, testQueue.getProcessed()) bq.maybeAdd(ctx, r, tc.store.Clock().NowAsClockTimestamp()) diff --git a/pkg/kv/kvserver/raft.go b/pkg/kv/kvserver/raft.go index 39c08bc4d22e..1c744c5fbd59 100644 --- a/pkg/kv/kvserver/raft.go +++ b/pkg/kv/kvserver/raft.go @@ -303,7 +303,6 @@ func traceProposals(r *Replica, ids []kvserverbase.CmdIDKey, event string) { ctxs = append(ctxs, prop.ctx) } } - // nolint:deferunlock r.mu.RUnlock() for _, ctx := range ctxs { log.Eventf(ctx, "%v", event) diff --git a/pkg/kv/kvserver/raft_log_queue.go b/pkg/kv/kvserver/raft_log_queue.go index 94fb5c37badd..639ccbe258c6 100644 --- a/pkg/kv/kvserver/raft_log_queue.go +++ b/pkg/kv/kvserver/raft_log_queue.go @@ -308,7 +308,6 @@ func newTruncateDecision(ctx context.Context, r *Replica) (truncateDecision, err }, ) log.Eventf(ctx, "raft status after lastUpdateTimes check: %+v", raftStatus.Progress) - // nolint:deferunlock r.mu.RUnlock() input := truncateDecisionInput{ @@ -698,10 +697,8 @@ func (rlq *raftLogQueue) process( r.mu.raftLogSize = n r.mu.raftLogLastCheckSize = n r.mu.raftLogSizeTrusted = true - // nolint:deferunlock r.mu.Unlock() } - // nolint:deferunlock r.raftMu.Unlock() if err != nil { diff --git a/pkg/kv/kvserver/raft_log_queue_test.go b/pkg/kv/kvserver/raft_log_queue_test.go index d6f28f821f84..0dc14944acd8 100644 --- a/pkg/kv/kvserver/raft_log_queue_test.go +++ b/pkg/kv/kvserver/raft_log_queue_test.go @@ -335,7 +335,6 @@ func verifyLogSizeInSync(t *testing.T, r *Replica) { defer r.raftMu.Unlock() r.mu.Lock() raftLogSize := r.mu.raftLogSize - // nolint:deferunlock r.mu.Unlock() actualRaftLogSize, err := ComputeRaftLogSize(context.Background(), r.RangeID, r.store.TODOEngine(), r.SideloadedRaftMuLocked()) if err != nil { @@ -739,7 +738,6 @@ func TestTruncateLog(t *testing.T) { // We can still get what remains of the log. tc.repl.mu.Lock() entries, err := tc.repl.raftEntriesLocked(indexes[5], indexes[9], math.MaxUint64) - // nolint:deferunlock tc.repl.mu.Unlock() if err != nil { t.Fatal(err) @@ -751,7 +749,6 @@ func TestTruncateLog(t *testing.T) { // But any range that includes the truncated entries returns an error. tc.repl.mu.Lock() _, err = tc.repl.raftEntriesLocked(indexes[4], indexes[9], math.MaxUint64) - // nolint:deferunlock tc.repl.mu.Unlock() if !errors.Is(err, raft.ErrCompacted) { t.Errorf("expected ErrCompacted, got %s", err) @@ -760,7 +757,6 @@ func TestTruncateLog(t *testing.T) { // The term of the last truncated entry is still available. tc.repl.mu.Lock() term, err := tc.repl.raftTermLocked(indexes[4]) - // nolint:deferunlock tc.repl.mu.Unlock() if err != nil { t.Fatal(err) @@ -772,7 +768,6 @@ func TestTruncateLog(t *testing.T) { // The terms of older entries are gone. tc.repl.mu.Lock() _, err = tc.repl.raftTermLocked(indexes[3]) - // nolint:deferunlock tc.repl.mu.Unlock() if !errors.Is(err, raft.ErrCompacted) { t.Errorf("expected ErrCompacted, got %s", err) @@ -795,7 +790,6 @@ func TestTruncateLog(t *testing.T) { tc.repl.mu.Lock() // The term of the last truncated entry is still available. term, err = tc.repl.raftTermLocked(indexes[4]) - // nolint:deferunlock tc.repl.mu.Unlock() if err != nil { t.Fatal(err) @@ -899,7 +893,6 @@ func TestTruncateLogRecompute(t *testing.T) { repl.mu.raftLogSizeTrusted = false repl.mu.raftLogSize += 12 // garbage repl.mu.raftLogLastCheckSize += 12 // garbage - // nolint:deferunlock repl.mu.Unlock() // Force a raft log queue run. The result should be a nonzero Raft log of diff --git a/pkg/kv/kvserver/raft_log_truncator.go b/pkg/kv/kvserver/raft_log_truncator.go index 36102508deb1..ace340843f1b 100644 --- a/pkg/kv/kvserver/raft_log_truncator.go +++ b/pkg/kv/kvserver/raft_log_truncator.go @@ -348,7 +348,6 @@ func (t *raftLogTruncator) addPendingTruncation( pendingTruncs.mu.Lock() // Install the new pending truncation. pendingTruncs.mu.truncs[pos] = pendingTrunc - // nolint:deferunlock pendingTruncs.mu.Unlock() if pos == 0 { @@ -393,7 +392,6 @@ func (t *raftLogTruncator) durabilityAdvancedCallback() { if !runTruncation && len(t.mu.addRanges) > 0 { t.mu.queuedDurabilityCB = true } - // nolint:deferunlock t.mu.Unlock() if !runTruncation { return @@ -410,7 +408,6 @@ func (t *raftLogTruncator) durabilityAdvancedCallback() { t.mu.runningTruncation = false shouldReturn = true } - // nolint:deferunlock t.mu.Unlock() if shouldReturn { return @@ -609,7 +606,6 @@ func (t *raftLogTruncator) tryEnactTruncations( for i := 0; i <= enactIndex; i++ { pendingTruncs.popLocked() } - // nolint:deferunlock pendingTruncs.mu.Unlock() if !pendingTruncs.isEmptyLocked() { t.enqueueRange(rangeID) @@ -619,6 +615,5 @@ func (t *raftLogTruncator) tryEnactTruncations( func (t *raftLogTruncator) enqueueRange(rangeID roachpb.RangeID) { t.mu.Lock() t.mu.addRanges[rangeID] = struct{}{} - // nolint:deferunlock t.mu.Unlock() } diff --git a/pkg/kv/kvserver/raft_transport.go b/pkg/kv/kvserver/raft_transport.go index 065a5ee38e57..0db457c010d8 100644 --- a/pkg/kv/kvserver/raft_transport.go +++ b/pkg/kv/kvserver/raft_transport.go @@ -492,7 +492,6 @@ func (t *RaftTransport) RaftMessageBatch(stream MultiRaft_RaftMessageBatchServer ctx := t.AnnotateCtx(context.Background()) t.kvflowControl.mu.Lock() t.kvflowControl.mu.connectionTracker.markStoresDisconnected(storeIDs) - // nolint:deferunlock t.kvflowControl.mu.Unlock() t.kvflowControl.disconnectListener.OnRaftTransportDisconnected(ctx, storeIDs...) if fn := t.knobs.OnServerStreamDisconnected; fn != nil { @@ -514,7 +513,6 @@ func (t *RaftTransport) RaftMessageBatch(stream MultiRaft_RaftMessageBatchServer } t.kvflowControl.mu.Lock() t.kvflowControl.mu.connectionTracker.markStoresConnected(storeIDs) - // nolint:deferunlock t.kvflowControl.mu.Unlock() if len(batch.Requests) == 0 { continue @@ -703,7 +701,6 @@ func (t *RaftTransport) processQueue( t.kvflowControl.mu.RLock() batch.StoreIDs = nil batch.StoreIDs = append(batch.StoreIDs, t.kvflowControl.mu.localStoreIDs...) - // nolint:deferunlock t.kvflowControl.mu.RUnlock() // Unconditionally set sentInitialStoreIDs, since we always have // the initial store IDs before the additional ones. @@ -869,7 +866,6 @@ func (t *RaftTransport) getQueue( } value, ok = queuesMap.LoadOrStore(int64(nodeID), unsafe.Pointer(&q)) t.kvflowControl.mu.connectionTracker.markNodeConnected(nodeID, class) - // nolint:deferunlock t.kvflowControl.mu.Unlock() } return (*raftSendQueue)(value), ok @@ -978,7 +974,6 @@ func (t *RaftTransport) startProcessNewQueue( t.kvflowControl.mu.Lock() t.queues[class].Delete(int64(toNodeID)) t.kvflowControl.mu.connectionTracker.markNodeDisconnected(toNodeID, class) - // nolint:deferunlock t.kvflowControl.mu.Unlock() }() conn, err := t.dialer.Dial(ctx, toNodeID, class) @@ -1009,7 +1004,6 @@ func (t *RaftTransport) startProcessNewQueue( t.kvflowControl.mu.Lock() t.queues[class].Delete(int64(toNodeID)) t.kvflowControl.mu.connectionTracker.markNodeDisconnected(toNodeID, class) - // nolint:deferunlock t.kvflowControl.mu.Unlock() return false } diff --git a/pkg/kv/kvserver/raftentry/cache.go b/pkg/kv/kvserver/raftentry/cache.go index 66c448eed640..10e6c53900d9 100644 --- a/pkg/kv/kvserver/raftentry/cache.go +++ b/pkg/kv/kvserver/raftentry/cache.go @@ -203,11 +203,9 @@ func (c *Cache) Clear(id roachpb.RangeID, hi kvpb.RaftIndex) { c.mu.Lock() p := c.getPartLocked(id, false /* create */, false /* recordUse */) if p == nil { - // nolint:deferunlock c.mu.Unlock() return } - // nolint:deferunlock c.mu.Unlock() p.mu.Lock() defer p.mu.Unlock() @@ -221,7 +219,6 @@ func (c *Cache) Get(id roachpb.RangeID, idx kvpb.RaftIndex) (e raftpb.Entry, ok c.metrics.Accesses.Inc(1) c.mu.Lock() p := c.getPartLocked(id, false /* create */, true /* recordUse */) - // nolint:deferunlock c.mu.Unlock() if p == nil { return e, false @@ -248,7 +245,6 @@ func (c *Cache) Scan( c.metrics.Accesses.Inc(1) c.mu.Lock() p := c.getPartLocked(id, false /* create */, true /* recordUse */) - // nolint:deferunlock c.mu.Unlock() if p == nil { return ents, 0, lo, false diff --git a/pkg/kv/kvserver/rangefeed/budget.go b/pkg/kv/kvserver/rangefeed/budget.go index 672f1f15d3a8..3d2d9f7b1294 100644 --- a/pkg/kv/kvserver/rangefeed/budget.go +++ b/pkg/kv/kvserver/rangefeed/budget.go @@ -212,14 +212,12 @@ func (f *FeedBudget) WaitAndGet( func (f *FeedBudget) returnAllocation(ctx context.Context, amount int64) { f.mu.Lock() if f.mu.closed { - // nolint:deferunlock f.mu.Unlock() return } if amount > 0 { f.mu.memBudget.Shrink(ctx, amount) } - // nolint:deferunlock f.mu.Unlock() select { case f.replenishC <- struct{}{}: @@ -238,7 +236,6 @@ func (f *FeedBudget) Close(ctx context.Context) { f.mu.closed = true f.mu.memBudget.Close(ctx) close(f.stopC) - // nolint:deferunlock f.mu.Unlock() }) } diff --git a/pkg/kv/kvserver/rangefeed/processor_test.go b/pkg/kv/kvserver/rangefeed/processor_test.go index 96a0f85dabf3..bf4257fcea16 100644 --- a/pkg/kv/kvserver/rangefeed/processor_test.go +++ b/pkg/kv/kvserver/rangefeed/processor_test.go @@ -1346,7 +1346,6 @@ func requireBudgetDrainedSoon(t *testing.T, p Processor, stream *consumer) { testutils.SucceedsSoon(t, func() error { processor.MemBudget.mu.Lock() used := processor.MemBudget.mu.memBudget.Used() - // nolint:deferunlock processor.MemBudget.mu.Unlock() fmt.Printf("Budget used: %d bytes, %d events processed\n", used, stream.Consumed()) diff --git a/pkg/kv/kvserver/rangefeed/registry.go b/pkg/kv/kvserver/rangefeed/registry.go index f11ca8957b8a..866fa852549f 100644 --- a/pkg/kv/kvserver/rangefeed/registry.go +++ b/pkg/kv/kvserver/rangefeed/registry.go @@ -337,7 +337,6 @@ func (r *registration) outputLoop(ctx context.Context) error { overflowed = r.mu.overflowed r.mu.caughtUp = true } - // nolint:deferunlock r.mu.Unlock() if overflowed { return newErrBufferCapacityExceeded().GoError() @@ -363,12 +362,10 @@ func (r *registration) runOutputLoop(ctx context.Context, _forStacks roachpb.Ran r.mu.Lock() if r.mu.disconnected { // The registration has already been disconnected. - // nolint:deferunlock r.mu.Unlock() return } ctx, r.mu.outputLoopCancelFn = context.WithCancel(ctx) - // nolint:deferunlock r.mu.Unlock() err := r.outputLoop(ctx) r.disconnect(kvpb.NewError(err)) @@ -583,7 +580,6 @@ func (r *registration) waitForCaughtUp() error { for re := retry.Start(opts); re.Next(); { r.mu.Lock() caughtUp := len(r.buf) == 0 && r.mu.caughtUp - // nolint:deferunlock r.mu.Unlock() if caughtUp { return nil diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index 5e4ad6fefb0c..30b468a56f02 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -190,7 +190,6 @@ func (mu *ReplicaMutex) Lock() { } func (mu *ReplicaMutex) Unlock() { - // nolint:deferunlock (*syncutil.RWMutex)(mu).Unlock() } @@ -207,7 +206,6 @@ func (mu *ReplicaMutex) AssertRHeld() { } func (mu *ReplicaMutex) RUnlock() { - // nolint:deferunlock (*syncutil.RWMutex)(mu).RUnlock() } @@ -1556,7 +1554,6 @@ func (r *Replica) State(ctx context.Context) kvserverpb.RangeInfo { r.sideTransportClosedTimestamp.mu.Lock() ri.ClosedTimestampSideTransportInfo.ReplicaClosed = r.sideTransportClosedTimestamp.mu.cur.ts ri.ClosedTimestampSideTransportInfo.ReplicaLAI = r.sideTransportClosedTimestamp.mu.cur.lai - // nolint:deferunlock r.sideTransportClosedTimestamp.mu.Unlock() centralClosed, centralLAI := r.store.cfg.ClosedTimestampReceiver.GetClosedTimestamp( ctx, r.RangeID, r.mu.state.Lease.Replica.NodeID) diff --git a/pkg/kv/kvserver/replica_app_batch.go b/pkg/kv/kvserver/replica_app_batch.go index c9767d4a908c..791783c16ae3 100644 --- a/pkg/kv/kvserver/replica_app_batch.go +++ b/pkg/kv/kvserver/replica_app_batch.go @@ -332,9 +332,7 @@ func (b *replicaAppBatch) runPostAddTriggersReplicaOnly( rhsRepl.mu.destroyStatus.Set( kvpb.NewRangeNotFoundError(rhsRepl.RangeID, rhsRepl.store.StoreID()), destroyReasonRemoved) - // nolint:deferunlock rhsRepl.mu.Unlock() - // nolint:deferunlock rhsRepl.readOnlyCmdMu.Unlock() // Use math.MaxInt32 (mergedTombstoneReplicaID) as the nextReplicaID as an @@ -445,7 +443,6 @@ func (b *replicaAppBatch) runPostAddTriggersReplicaOnly( // !looselyCoupledTruncation code path. b.r.mu.Lock() b.r.mu.raftLogSizeTrusted = false - // nolint:deferunlock b.r.mu.Unlock() } } @@ -475,9 +472,7 @@ func (b *replicaAppBatch) runPostAddTriggersReplicaOnly( kvpb.NewRangeNotFoundError(b.r.RangeID, b.r.store.StoreID()), destroyReasonRemoved) span := b.r.descRLocked().RSpan() - // nolint:deferunlock b.r.mu.Unlock() - // nolint:deferunlock b.r.readOnlyCmdMu.Unlock() b.changeRemovesReplica = true @@ -585,7 +580,6 @@ func (b *replicaAppBatch) ApplyToStateMachine(ctx context.Context) error { existingClosed := r.mu.state.RaftClosedTimestamp newClosed := b.state.RaftClosedTimestamp if !newClosed.IsEmpty() && newClosed.Less(existingClosed) && raftClosedTimestampAssertionsEnabled { - // nolint:deferunlock r.mu.Unlock() return errors.AssertionFailedf( "raft closed timestamp regression; replica has: %s, new batch has: %s.", diff --git a/pkg/kv/kvserver/replica_application_result.go b/pkg/kv/kvserver/replica_application_result.go index ea423124b5ea..5a5699e0909c 100644 --- a/pkg/kv/kvserver/replica_application_result.go +++ b/pkg/kv/kvserver/replica_application_result.go @@ -478,7 +478,6 @@ func (r *Replica) handleTruncatedStateResult( expectedFirstIndexWasAccurate = r.mu.state.TruncatedState.Index+1 == expectedFirstIndexPreTruncation r.mu.state.TruncatedState = t - // nolint:deferunlock r.mu.Unlock() // Clear any entries in the Raft log entry cache for this range up @@ -507,14 +506,12 @@ func (r *Replica) handleGCThresholdResult(ctx context.Context, thresh *hlc.Times } r.mu.Lock() r.mu.state.GCThreshold = thresh - // nolint:deferunlock r.mu.Unlock() } func (r *Replica) handleGCHintResult(ctx context.Context, hint *roachpb.GCHint) { r.mu.Lock() r.mu.state.GCHint = hint - // nolint:deferunlock r.mu.Unlock() } @@ -524,7 +521,6 @@ func (r *Replica) handleVersionResult(ctx context.Context, version *roachpb.Vers } r.mu.Lock() r.mu.state.Version = version - // nolint:deferunlock r.mu.Unlock() } diff --git a/pkg/kv/kvserver/replica_application_state_machine.go b/pkg/kv/kvserver/replica_application_state_machine.go index df6b761f2d90..73fa68a6a140 100644 --- a/pkg/kv/kvserver/replica_application_state_machine.go +++ b/pkg/kv/kvserver/replica_application_state_machine.go @@ -128,7 +128,6 @@ func (sm *replicaStateMachine) NewEphemeralBatch() apply.EphemeralBatch { mb.r = r r.mu.RLock() mb.state = r.mu.state - // nolint:deferunlock r.mu.RUnlock() return mb } @@ -145,7 +144,6 @@ func (sm *replicaStateMachine) NewBatch() apply.Batch { b.state.Stats = &sm.stats *b.state.Stats = *r.mu.state.Stats b.closedTimestampSetter = r.mu.closedTimestampSetter - // nolint:deferunlock r.mu.RUnlock() b.start = timeutil.Now() return b @@ -202,7 +200,6 @@ func (sm *replicaStateMachine) ApplySideEffects( // TODO(sep-raft-log): either check only statemachine invariants or // pass both engines in. sm.r.assertStateRaftMuLockedReplicaMuRLocked(ctx, sm.r.store.TODOEngine()) - // nolint:deferunlock sm.r.mu.RUnlock() sm.applyStats.stateAssertions++ } @@ -248,7 +245,6 @@ func (sm *replicaStateMachine) ApplySideEffects( if higherReproposalsExist { sm.r.mu.Lock() delete(sm.r.mu.proposals, cmd.ID) - // nolint:deferunlock sm.r.mu.Unlock() } cmd.proposal.applied = true diff --git a/pkg/kv/kvserver/replica_application_state_machine_test.go b/pkg/kv/kvserver/replica_application_state_machine_test.go index e28279591723..a134895f9582 100644 --- a/pkg/kv/kvserver/replica_application_state_machine_test.go +++ b/pkg/kv/kvserver/replica_application_state_machine_test.go @@ -188,7 +188,6 @@ func TestReplicaStateMachineRaftLogTruncationStronglyCoupled(t *testing.T) { // Overwrite to be trusted, since we want to check if transitions to false // or not. r.mu.raftLogSizeTrusted = true - // nolint:deferunlock r.mu.Unlock() expectedFirstIndex := truncatedIndex + 1 @@ -302,7 +301,6 @@ func TestReplicaStateMachineRaftLogTruncationLooselyCoupled(t *testing.T) { // Overwrite to be trusted, since we want to check if transitions to false // or not. r.mu.raftLogSizeTrusted = true - // nolint:deferunlock r.mu.Unlock() expectedFirstIndex := truncatedIndex + 1 if !accurate { @@ -426,7 +424,6 @@ func TestReplicaStateMachineEphemeralAppBatchRejection(t *testing.T) { r.mu.Lock() raftAppliedIndex := r.mu.state.RaftAppliedIndex - // nolint:deferunlock r.mu.Unlock() descWriteRepr := func(v string) (kvpb.Request, []byte) { diff --git a/pkg/kv/kvserver/replica_closedts.go b/pkg/kv/kvserver/replica_closedts.go index e68700a313c9..1e9bce62999c 100644 --- a/pkg/kv/kvserver/replica_closedts.go +++ b/pkg/kv/kvserver/replica_closedts.go @@ -287,7 +287,6 @@ func (st *sidetransportAccess) get( ) hlc.Timestamp { st.mu.RLock() cur, next := st.mu.cur, st.mu.next - // nolint:deferunlock st.mu.RUnlock() // If the current info is enough to satisfy sufficient, we're done. diff --git a/pkg/kv/kvserver/replica_closedts_internal_test.go b/pkg/kv/kvserver/replica_closedts_internal_test.go index fee98c8322a3..d7a83f22f2ce 100644 --- a/pkg/kv/kvserver/replica_closedts_internal_test.go +++ b/pkg/kv/kvserver/replica_closedts_internal_test.go @@ -413,14 +413,12 @@ func TestSideTransportClosedMonotonic(t *testing.T) { truth.lai++ } cur := truth.closedTimestamp - // nolint:deferunlock truth.Unlock() // Optionally update receiver. if rand.Intn(2) == 0 { r.Lock() r.closedTimestamp = cur - // nolint:deferunlock r.Unlock() } @@ -430,7 +428,6 @@ func TestSideTransportClosedMonotonic(t *testing.T) { r.Lock() s.forward(ctx, r.ts, r.lai, knownApplied) r.closedTimestamp = closedTimestamp{} - // nolint:deferunlock r.Unlock() } } @@ -457,7 +454,6 @@ func TestSideTransportClosedMonotonic(t *testing.T) { case 2: truth.Lock() lai = truth.lai - // nolint:deferunlock truth.Unlock() } @@ -686,7 +682,6 @@ func TestQueryResolvedTimestamp(t *testing.T) { // Inject a closed timestamp. tc.repl.mu.Lock() tc.repl.mu.state.RaftClosedTimestamp = test.closedTS - // nolint:deferunlock tc.repl.mu.Unlock() // Issue a QueryResolvedTimestamp request. @@ -751,7 +746,6 @@ func TestQueryResolvedTimestampResolvesAbandonedIntents(t *testing.T) { tc.manualClock.AdvanceTo(ts20.GoTime()) tc.repl.mu.Lock() tc.repl.mu.state.RaftClosedTimestamp = ts20 - // nolint:deferunlock tc.repl.mu.Unlock() // Issue a QueryResolvedTimestamp request. Should return resolved timestamp @@ -991,7 +985,6 @@ func TestServerSideBoundedStalenessNegotiation(t *testing.T) { // Inject a closed timestamp. tc.repl.mu.Lock() tc.repl.mu.state.RaftClosedTimestamp = closedTS - // nolint:deferunlock tc.repl.mu.Unlock() // Construct and issue the request. @@ -1086,7 +1079,6 @@ func TestServerSideBoundedStalenessNegotiationWithResumeSpan(t *testing.T) { // Inject a closed timestamp. tc.repl.mu.Lock() tc.repl.mu.state.RaftClosedTimestamp = makeTS(30) - // nolint:deferunlock tc.repl.mu.Unlock() // Return the timestamp of the earliest intent. diff --git a/pkg/kv/kvserver/replica_command.go b/pkg/kv/kvserver/replica_command.go index cfdefe7e2856..5bd28858c209 100644 --- a/pkg/kv/kvserver/replica_command.go +++ b/pkg/kv/kvserver/replica_command.go @@ -1001,7 +1001,6 @@ func (r *Replica) ChangeReplicas( bq := r.store.replicateQueue.baseQueue bq.mu.Lock() disabled := bq.mu.disabled - // nolint:deferunlock bq.mu.Unlock() if !disabled { return nil, errors.New("must disable replicate queue to use ChangeReplicas manually") @@ -2789,7 +2788,6 @@ func (r *Replica) sendSnapshotUsingDelegate( r.mu.RLock() sender, err := r.getReplicaDescriptorRLocked() _, destPaused := r.mu.pausedFollowers[recipient.ReplicaID] - // nolint:deferunlock r.mu.RUnlock() if err != nil { @@ -2972,7 +2970,6 @@ func (r *Replica) validateSnapshotDelegationRequest( r.mu.RLock() replIdx := r.mu.state.RaftAppliedIndex + 1 status := r.raftStatusRLocked() - // nolint:deferunlock r.mu.RUnlock() if status == nil { diff --git a/pkg/kv/kvserver/replica_learner_test.go b/pkg/kv/kvserver/replica_learner_test.go index f2a45504ecf9..bb6a84144b1b 100644 --- a/pkg/kv/kvserver/replica_learner_test.go +++ b/pkg/kv/kvserver/replica_learner_test.go @@ -563,7 +563,6 @@ func TestDelegateSnapshotFails(t *testing.T) { // Always use node 3 (index 2) as the only delegate. senders.mu.Lock() senders.desc = append(senders.desc, roachpb.ReplicaDescriptor{NodeID: 3, StoreID: 3}) - // nolint:deferunlock senders.mu.Unlock() // Now stop accepting traffic to node 3 (index 2). @@ -597,7 +596,6 @@ func TestDelegateSnapshotFails(t *testing.T) { senders.mu.Lock() senders.desc = append(senders.desc, roachpb.ReplicaDescriptor{NodeID: 3, StoreID: 3}) senders.desc = append(senders.desc, roachpb.ReplicaDescriptor{NodeID: 1, StoreID: 1}) - // nolint:deferunlock senders.mu.Unlock() // Now stop accepting traffic to node 3 (index 2). @@ -643,7 +641,6 @@ func TestDelegateSnapshotFails(t *testing.T) { senders.mu.Lock() senders.desc = append(senders.desc, roachpb.ReplicaDescriptor{NodeID: 3, StoreID: 3}) senders.desc = append(senders.desc, roachpb.ReplicaDescriptor{NodeID: 1, StoreID: 1}) - // nolint:deferunlock senders.mu.Unlock() block.Store(2) @@ -671,7 +668,6 @@ func TestDelegateSnapshotFails(t *testing.T) { senders.mu.Lock() senders.desc = append(senders.desc, roachpb.ReplicaDescriptor{NodeID: 4, StoreID: 4}) senders.desc = append(senders.desc, roachpb.ReplicaDescriptor{NodeID: 1, StoreID: 1}) - // nolint:deferunlock senders.mu.Unlock() _, err := tc.Servers[0].DB().AdminChangeReplicas( @@ -704,7 +700,6 @@ func TestDelegateSnapshotFails(t *testing.T) { // Choose the store which we are about to block. senders.mu.Lock() senders.desc = append(senders.desc, roachpb.ReplicaDescriptor{NodeID: 4, StoreID: 4}) - // nolint:deferunlock senders.mu.Unlock() // Don't allow store 4 to see the new descriptor through Raft. @@ -753,7 +748,6 @@ func TestDelegateSnapshotFails(t *testing.T) { senders.mu.Lock() senders.desc = append(senders.desc, roachpb.ReplicaDescriptor{NodeID: 4, StoreID: 4}) senders.desc = append(senders.desc, roachpb.ReplicaDescriptor{NodeID: 1, StoreID: 1}) - // nolint:deferunlock senders.mu.Unlock() // First try to use node 3 (index 2) as the delegate, but fall back to the leaseholder on failure. diff --git a/pkg/kv/kvserver/replica_metrics.go b/pkg/kv/kvserver/replica_metrics.go index c1e54d642064..84ae566d2a07 100644 --- a/pkg/kv/kvserver/replica_metrics.go +++ b/pkg/kv/kvserver/replica_metrics.go @@ -71,7 +71,6 @@ func (r *Replica) Metrics( ) ReplicaMetrics { r.store.unquiescedReplicas.Lock() _, ticking := r.store.unquiescedReplicas.m[r.RangeID] - // nolint:deferunlock r.store.unquiescedReplicas.Unlock() latchMetrics := r.concMgr.LatchMetrics() diff --git a/pkg/kv/kvserver/replica_probe_test.go b/pkg/kv/kvserver/replica_probe_test.go index e66cf5c51e74..7dae73451327 100644 --- a/pkg/kv/kvserver/replica_probe_test.go +++ b/pkg/kv/kvserver/replica_probe_test.go @@ -187,7 +187,6 @@ func TestReplicaProbeRequest(t *testing.T) { injErr := kvpb.NewErrorf("bang") seen.Lock() seen.injectedErr = injErr - // nolint:deferunlock seen.Unlock() for _, srv := range tc.Servers { repl, _, err := srv.GetStores().(*kvserver.Stores).GetReplicaForRangeID(ctx, desc.RangeID) diff --git a/pkg/kv/kvserver/replica_proposal_buf_test.go b/pkg/kv/kvserver/replica_proposal_buf_test.go index b1568ac73726..2f58569658b1 100644 --- a/pkg/kv/kvserver/replica_proposal_buf_test.go +++ b/pkg/kv/kvserver/replica_proposal_buf_test.go @@ -472,7 +472,6 @@ func TestProposalBufferConcurrentWithDestroy(t *testing.T) { // Destroy the proposer. All producers and consumers should notice. p.Lock() p.ds.Set(dsErr, destroyReasonRemoved) - // nolint:deferunlock p.Unlock() require.Nil(t, g.Wait()) diff --git a/pkg/kv/kvserver/replica_proposal_quota.go b/pkg/kv/kvserver/replica_proposal_quota.go index 2b430d527ebd..cece7b811f22 100644 --- a/pkg/kv/kvserver/replica_proposal_quota.go +++ b/pkg/kv/kvserver/replica_proposal_quota.go @@ -39,7 +39,6 @@ func (r *Replica) maybeAcquireProposalQuota( r.mu.RLock() quotaPool := r.mu.proposalQuota desc := *r.mu.state.Desc - // nolint:deferunlock r.mu.RUnlock() // Quota acquisition only takes place on the leader replica, diff --git a/pkg/kv/kvserver/replica_protected_timestamp_test.go b/pkg/kv/kvserver/replica_protected_timestamp_test.go index 1b967211874d..45ecc796ffe6 100644 --- a/pkg/kv/kvserver/replica_protected_timestamp_test.go +++ b/pkg/kv/kvserver/replica_protected_timestamp_test.go @@ -108,7 +108,6 @@ func TestCheckProtectedTimestampsForGC(t *testing.T) { test: func(t *testing.T, r *Replica, mp *manualPTSReader) { r.mu.Lock() th := *r.mu.state.GCThreshold - // nolint:deferunlock r.mu.Unlock() mp.asOf = r.store.Clock().Now().Next() mp.protections = append(mp.protections, manualPTSReaderProtection{ @@ -174,7 +173,6 @@ func TestCheckProtectedTimestampsForGC(t *testing.T) { r.mu.Lock() r.mu.state.GCThreshold = &tsMinus60s r.mu.state.Lease.Start = ts.UnsafeToClockTimestamp() - // nolint:deferunlock r.mu.Unlock() canGC, readAt, gcTimestamp, oldThreshold, newThreshold, err := r.checkProtectedTimestampsForGC(ctx, makeTTLDuration(gcTTLSec)) diff --git a/pkg/kv/kvserver/replica_raft.go b/pkg/kv/kvserver/replica_raft.go index 3d5b9718094c..fe71cf3ac37f 100644 --- a/pkg/kv/kvserver/replica_raft.go +++ b/pkg/kv/kvserver/replica_raft.go @@ -965,7 +965,6 @@ func (r *Replica) handleRaftReadyRaftMuLocked( LastTerm: r.mu.lastTermNotDurable, ByteSize: r.mu.raftLogSize, } - // nolint:deferunlock r.mu.RUnlock() // We refresh pending commands after applying a snapshot because this @@ -1098,7 +1097,6 @@ func (r *Replica) handleRaftReadyRaftMuLocked( if refreshReason != noReason { r.mu.Lock() r.refreshProposalsLocked(ctx, 0 /* refreshAtDelta */, refreshReason) - // nolint:deferunlock r.mu.Unlock() } @@ -1257,7 +1255,6 @@ func (r *Replica) tick( r.unreachablesMu.Lock() remotes := r.unreachablesMu.remotes r.unreachablesMu.remotes = nil - // nolint:deferunlock r.unreachablesMu.Unlock() for remoteReplica := range remotes { r.mu.internalRaftGroup.ReportUnreachable(uint64(remoteReplica)) @@ -1697,7 +1694,6 @@ func (r *Replica) sendLocalRaftMsg(msg raftpb.Message, willDeliverLocal bool) { r.localMsgs.Lock() wasEmpty := len(r.localMsgs.active) == 0 r.localMsgs.active = append(r.localMsgs.active, msg) - // nolint:deferunlock r.localMsgs.Unlock() // If this is the first local message and the caller will not deliver local // messages itself, schedule a Raft update check to inform Raft processing @@ -1722,7 +1718,6 @@ func (r *Replica) deliverLocalRaftMsgsRaftMuLockedReplicaMuLocked( if cap(r.localMsgs.recycled) > 16 { r.localMsgs.recycled = nil } - // nolint:deferunlock r.localMsgs.Unlock() // If we are in a test build, shuffle the local messages before delivering @@ -1809,7 +1804,6 @@ func (r *Replica) sendRaftMessage(ctx context.Context, msg raftpb.Message) { if !r.sendRaftMessageRequest(ctx, req) { r.mu.Lock() r.mu.droppedMessages++ - // nolint:deferunlock r.mu.Unlock() r.addUnreachableRemoteReplica(toReplica.ReplicaID) } @@ -1823,7 +1817,6 @@ func (r *Replica) addUnreachableRemoteReplica(remoteReplica roachpb.ReplicaID) { r.unreachablesMu.remotes = make(map[roachpb.ReplicaID]struct{}) } r.unreachablesMu.remotes[remoteReplica] = struct{}{} - // nolint:deferunlock r.unreachablesMu.Unlock() } @@ -2449,7 +2442,6 @@ func (r *Replica) maybeAcquireSnapshotMergeLock( } return subsumedRepls, func() { for _, sr := range subsumedRepls { - // nolint:deferunlock sr.raftMu.Unlock() } } diff --git a/pkg/kv/kvserver/replica_raft_quiesce.go b/pkg/kv/kvserver/replica_raft_quiesce.go index c8aa5f05ab95..3a9eaa861e52 100644 --- a/pkg/kv/kvserver/replica_raft_quiesce.go +++ b/pkg/kv/kvserver/replica_raft_quiesce.go @@ -46,7 +46,6 @@ func (r *Replica) quiesceLocked(ctx context.Context, lagging laggingReplicaSet) r.mu.laggingFollowersOnQuiesce = lagging r.store.unquiescedReplicas.Lock() delete(r.store.unquiescedReplicas.m, r.RangeID) - // nolint:deferunlock r.store.unquiescedReplicas.Unlock() } else if log.V(4) { log.Infof(ctx, "r%d already quiesced", r.RangeID) @@ -90,7 +89,6 @@ func (r *Replica) maybeUnquiesceLocked(wakeLeader, mayCampaign bool) bool { r.mu.laggingFollowersOnQuiesce = nil r.store.unquiescedReplicas.Lock() r.store.unquiescedReplicas.m[r.RangeID] = struct{}{} - // nolint:deferunlock r.store.unquiescedReplicas.Unlock() st := r.raftSparseStatusRLocked() diff --git a/pkg/kv/kvserver/replica_raftstorage.go b/pkg/kv/kvserver/replica_raftstorage.go index e52c01250ed0..d53fd1c58d93 100644 --- a/pkg/kv/kvserver/replica_raftstorage.go +++ b/pkg/kv/kvserver/replica_raftstorage.go @@ -237,7 +237,6 @@ func (r *Replica) GetSnapshot( // the corresponding Raft command not applied yet). r.raftMu.Lock() snap := r.store.TODOEngine().NewSnapshot() - // nolint:deferunlock r.raftMu.Unlock() defer func() { @@ -560,9 +559,7 @@ func (r *Replica) applySnapshot( sr.mu.destroyStatus.Set( kvpb.NewRangeNotFoundError(sr.RangeID, sr.store.StoreID()), destroyReasonRemoved) - // nolint:deferunlock sr.mu.Unlock() - // nolint:deferunlock sr.readOnlyCmdMu.Unlock() subsumedDescs = append(subsumedDescs, sr.Desc()) @@ -668,11 +665,9 @@ func (r *Replica) applySnapshot( if isInitialSnap { // NB: this will also call setDescLockedRaftMuLocked. if err := r.initFromSnapshotLockedRaftMuLocked(ctx, desc); err != nil { - // nolint:deferunlock r.mu.Unlock() log.Fatalf(ctx, "unable to initialize replica while applying snapshot: %+v", err) } else if err := r.store.markReplicaInitializedLockedReplLocked(ctx, r); err != nil { - // nolint:deferunlock r.mu.Unlock() log.Fatalf(ctx, "unable to mark replica initialized while applying snapshot: %+v", err) } @@ -737,7 +732,6 @@ func (r *Replica) applySnapshot( // across both Replica.mu critical sections. r.mu.RLock() r.assertStateRaftMuLockedReplicaMuRLocked(ctx, r.store.TODOEngine()) - // nolint:deferunlock r.mu.RUnlock() // The rangefeed processor is listening for the logical ops attached to diff --git a/pkg/kv/kvserver/replica_range_lease.go b/pkg/kv/kvserver/replica_range_lease.go index 03d95bf589fb..4fe38b618491 100644 --- a/pkg/kv/kvserver/replica_range_lease.go +++ b/pkg/kv/kvserver/replica_range_lease.go @@ -1527,7 +1527,6 @@ func (r *Replica) maybeSwitchLeaseType(ctx context.Context, st kvserverpb.LeaseS if !r.hasCorrectLeaseTypeRLocked(st.Lease) { llHandle = r.requestLeaseLocked(ctx, st, nil /* limiter */) } - // nolint:deferunlock r.mu.Unlock() if llHandle != nil { diff --git a/pkg/kv/kvserver/replica_rangefeed.go b/pkg/kv/kvserver/replica_rangefeed.go index 319569bcb825..7def586f7dc3 100644 --- a/pkg/kv/kvserver/replica_rangefeed.go +++ b/pkg/kv/kvserver/replica_rangefeed.go @@ -237,7 +237,6 @@ func (r *Replica) RangeFeed( // the registration doesn't miss any events. r.raftMu.Lock() if err := r.checkExecutionCanProceedForRangeFeed(ctx, rSpan, checkTS); err != nil { - // nolint:deferunlock r.raftMu.Unlock() iterSemRelease() return future.MakeCompletedErrorFuture(err) @@ -370,7 +369,6 @@ func (r *Replica) registerWithRangefeedRaftMuLocked( // Update the rangefeed filter to avoid filtering ops // that this new registration might be interested in. r.setRangefeedFilterLocked(filter) - // nolint:deferunlock r.rangefeedMu.Unlock() return p } @@ -452,7 +450,6 @@ func (r *Replica) registerWithRangefeedRaftMuLocked( r.setRangefeedProcessor(p) r.rangefeedMu.Lock() r.setRangefeedFilterLocked(filter) - // nolint:deferunlock r.rangefeedMu.Unlock() // Check for an initial closed timestamp update immediately to help diff --git a/pkg/kv/kvserver/replica_rankings.go b/pkg/kv/kvserver/replica_rankings.go index 857fa39c506d..b7f13dacd444 100644 --- a/pkg/kv/kvserver/replica_rankings.go +++ b/pkg/kv/kvserver/replica_rankings.go @@ -122,7 +122,6 @@ func NewReplicaAccumulator(dims ...load.Dimension) *RRAccumulator { func (rr *ReplicaRankings) Update(acc *RRAccumulator) { rr.mu.Lock() rr.mu.dimAccumulator = acc - // nolint:deferunlock rr.mu.Unlock() } @@ -252,7 +251,6 @@ func NewTenantReplicaAccumulator(dims ...load.Dimension) *RRAccumulatorByTenant func (rr *ReplicaRankingMap) Update(acc *RRAccumulatorByTenant) { rr.mu.Lock() rr.mu.dimAccumulators = acc - // nolint:deferunlock rr.mu.Unlock() } diff --git a/pkg/kv/kvserver/replica_send.go b/pkg/kv/kvserver/replica_send.go index 64a9045fe54d..1aa422ac942b 100644 --- a/pkg/kv/kvserver/replica_send.go +++ b/pkg/kv/kvserver/replica_send.go @@ -1117,7 +1117,6 @@ func (r *Replica) collectSpans( r.mu.RLock() desc := r.descRLocked() liveCount := r.mu.state.Stats.LiveCount - // nolint:deferunlock r.mu.RUnlock() // TODO(bdarnell): need to make this less global when local // latches are used more heavily. For example, a split will diff --git a/pkg/kv/kvserver/replica_sst_snapshot_storage.go b/pkg/kv/kvserver/replica_sst_snapshot_storage.go index 445e53dbc732..d7ce8c6cc12f 100644 --- a/pkg/kv/kvserver/replica_sst_snapshot_storage.go +++ b/pkg/kv/kvserver/replica_sst_snapshot_storage.go @@ -62,7 +62,6 @@ func (s *SSTSnapshotStorage) NewScratchSpace( ) *SSTSnapshotStorageScratch { s.mu.Lock() s.mu.rangeRefCount[rangeID]++ - // nolint:deferunlock s.mu.Unlock() snapDir := filepath.Join(s.dir, strconv.Itoa(int(rangeID)), snapUUID.String()) return &SSTSnapshotStorageScratch{ diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index ab0bd17b011b..e2890c001a18 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -304,9 +304,7 @@ func (tc *testContext) addBogusReplicaToRangeDesc( tc.repl.raftMu.Lock() tc.repl.mu.RLock() tc.repl.assertStateRaftMuLockedReplicaMuRLocked(ctx, tc.engine) - // nolint:deferunlock tc.repl.mu.RUnlock() - // nolint:deferunlock tc.repl.raftMu.Unlock() return newReplica, nil } @@ -713,12 +711,10 @@ func TestBehaviorDuringLeaseTransfer(t *testing.T) { tc.repl.mu.Lock() repDesc, err := tc.repl.getReplicaDescriptorRLocked() if err != nil { - // nolint:deferunlock tc.repl.mu.Unlock() t.Fatal(err) } pending := tc.repl.mu.pendingLeaseRequest.TransferInProgress(repDesc.ReplicaID) - // nolint:deferunlock tc.repl.mu.Unlock() if !pending { t.Fatalf("expected transfer to be in progress, and it wasn't") @@ -866,7 +862,6 @@ func TestLeaseReplicaNotInDesc(t *testing.T) { &tc.repl.mu.state, ) pErr := fr.ForcedError - // nolint:deferunlock tc.repl.mu.Unlock() if _, isErr := pErr.GetDetail().(*kvpb.LeaseRejectedError); !isErr { t.Fatal(pErr) @@ -1908,7 +1903,6 @@ func TestAcquireLease(t *testing.T) { } else { expStart = tc.repl.mu.minLeaseProposedTS } - // nolint:deferunlock tc.repl.mu.Unlock() tc.manualClock.MustAdvanceTo(leaseExpiry(tc.repl)) @@ -2007,7 +2001,6 @@ func TestLeaseConcurrent(t *testing.T) { tc.repl.mu.Lock() status := tc.repl.leaseStatusAtRLocked(ctx, now) llHandle := tc.repl.requestLeaseLocked(ctx, status, nil) - // nolint:deferunlock tc.repl.mu.Unlock() wg.Done() pErr := <-llHandle.C() @@ -2106,7 +2099,6 @@ func TestLeaseCallerCancelled(t *testing.T) { tc.repl.mu.Lock() status := tc.repl.leaseStatusAtRLocked(ctx, now) llHandles = append(llHandles, tc.repl.requestLeaseLocked(ctx, status, nil)) - // nolint:deferunlock tc.repl.mu.Unlock() } for _, llHandle := range llHandles { @@ -2146,7 +2138,6 @@ func TestRequestLeaseLimit(t *testing.T) { tc.repl.mu.Lock() status := tc.repl.leaseStatusAtRLocked(ctx, now) llHandle := tc.repl.requestLeaseLocked(ctx, status, limiter) - // nolint:deferunlock tc.repl.mu.Unlock() pErr := <-llHandle.C() return pErr.GoError() @@ -6673,7 +6664,6 @@ func TestAppliedIndex(t *testing.T) { tc.repl.mu.Lock() newAppliedIndex := tc.repl.mu.state.RaftAppliedIndex - // nolint:deferunlock tc.repl.mu.Unlock() if newAppliedIndex <= appliedIndex { t.Errorf("appliedIndex did not advance. Was %d, now %d", appliedIndex, newAppliedIndex) @@ -7426,7 +7416,6 @@ func TestEntries(t *testing.T) { } repl.mu.Lock() ents, err := repl.raftEntriesLocked(tc.lo, tc.hi, tc.maxBytes) - // nolint:deferunlock repl.mu.Unlock() if tc.expError == nil && err != nil { t.Errorf("%d: expected no error, got %s", i, err) @@ -7450,7 +7439,6 @@ func TestEntries(t *testing.T) { if _, err := repl.raftEntriesLocked(indexes[9], indexes[5], math.MaxUint64); err == nil { t.Errorf("23: error expected, got none") } - // nolint:deferunlock repl.mu.Unlock() }) } @@ -7700,7 +7688,6 @@ func TestReplicaAbandonProposal(t *testing.T) { } return false, nil } - // nolint:deferunlock tc.repl.mu.Unlock() ba := &kvpb.BatchRequest{} @@ -7773,7 +7760,6 @@ func TestSyncSnapshot(t *testing.T) { // first try. tc.repl.mu.Lock() snap, err := tc.repl.raftSnapshotLocked() - // nolint:deferunlock tc.repl.mu.Unlock() if err != nil { @@ -7810,7 +7796,6 @@ func TestReplicaRetryRaftProposal(t *testing.T) { } return 0 } - // nolint:deferunlock tc.repl.mu.Unlock() pArg := putArgs(roachpb.Key("a"), []byte("asd")) @@ -7830,7 +7815,6 @@ func TestReplicaRetryRaftProposal(t *testing.T) { if wrongLeaseIndex < 1 { t.Fatal("committed a few batches, but still at lease index zero") } - // nolint:deferunlock tc.repl.mu.RUnlock() log.Infof(ctx, "test begins") @@ -7915,7 +7899,6 @@ func TestReplicaCancelRaftCommandProgress(t *testing.T) { } return false, nil } - // nolint:deferunlock tc.repl.mu.Unlock() var chs []chan proposalResult @@ -7941,7 +7924,6 @@ func TestReplicaCancelRaftCommandProgress(t *testing.T) { } else { chs = append(chs, ch) } - // nolint:deferunlock repl.mu.Unlock() } @@ -7989,7 +7971,6 @@ func TestReplicaBurstPendingCommandsAndRepropose(t *testing.T) { } return false, nil } - // nolint:deferunlock tc.repl.mu.Unlock() const num = 10 @@ -8031,9 +8012,7 @@ func TestReplicaBurstPendingCommandsAndRepropose(t *testing.T) { if err := tc.repl.mu.proposalBuf.flushLocked(ctx); err != nil { t.Fatal(err) } - // nolint:deferunlock tc.repl.mu.Unlock() - // nolint:deferunlock tc.repl.raftMu.Unlock() for _, ch := range chs { @@ -8086,7 +8065,6 @@ func TestReplicaRefreshPendingCommandsTicks(t *testing.T) { // starting with a value of 0 modulo reproposalTicks. r.mu.Lock() ticks := r.mu.ticks - // nolint:deferunlock r.mu.Unlock() for ; (ticks % reproposalTicks) != 0; ticks++ { if _, err := r.tick(ctx, nil, nil); err != nil { @@ -8108,7 +8086,6 @@ func TestReplicaRefreshPendingCommandsTicks(t *testing.T) { _, ok := dropProposals.m[p] return ok, nil } - // nolint:deferunlock r.mu.Unlock() // We tick the replica 3*RaftReproposalTimeoutTicks. @@ -8126,7 +8103,6 @@ func TestReplicaRefreshPendingCommandsTicks(t *testing.T) { dropProposals.Lock() dropProposals.m[cmd] = struct{}{} // silently drop proposals - // nolint:deferunlock dropProposals.Unlock() cmd.command.ProposerLeaseSequence = st.Lease.Sequence @@ -8138,7 +8114,6 @@ func TestReplicaRefreshPendingCommandsTicks(t *testing.T) { if err := tc.repl.mu.proposalBuf.flushLocked(ctx); err != nil { t.Fatal(err) } - // nolint:deferunlock r.mu.Unlock() // Tick raft. @@ -8149,7 +8124,6 @@ func TestReplicaRefreshPendingCommandsTicks(t *testing.T) { r.mu.Lock() ticks := r.mu.ticks - // nolint:deferunlock r.mu.Unlock() var reproposed []*ProposalData @@ -8163,7 +8137,6 @@ func TestReplicaRefreshPendingCommandsTicks(t *testing.T) { reproposed = append(reproposed, p) } } - // nolint:deferunlock dropProposals.Unlock() r.mu.Unlock() @@ -9497,7 +9470,6 @@ func TestNoopRequestsNotProposed(t *testing.T) { } return nil } - // nolint:deferunlock repl.mu.Unlock() ba := &kvpb.BatchRequest{} @@ -10323,9 +10295,7 @@ func TestReplicaRecomputeStats(t *testing.T) { ms.Add(*disturbMS) err := repl.raftMu.stateLoader.SetMVCCStats(ctx, tc.engine, ms) repl.assertStateRaftMuLockedReplicaMuRLocked(ctx, tc.engine) - // nolint:deferunlock repl.mu.Unlock() - // nolint:deferunlock repl.raftMu.Unlock() if err != nil { @@ -13486,7 +13456,6 @@ func TestReplicateQueueProcessOne(t *testing.T) { errBoom := errors.New("boom") tc.repl.mu.Lock() tc.repl.mu.destroyStatus.Set(errBoom, destroyReasonMergePending) - // nolint:deferunlock tc.repl.mu.Unlock() requeue, err := tc.store.replicateQueue.processOneChange( @@ -13723,7 +13692,6 @@ func TestRangeInfoReturned(t *testing.T) { tc.repl.mu.Lock() st := tc.repl.leaseStatusAtRLocked(ctx, tc.Clock().NowAsClockTimestamp()) ll := tc.repl.requestLeaseLocked(ctx, st, nil /* limiter */) - // nolint:deferunlock tc.repl.mu.Unlock() select { case pErr := <-ll.C(): diff --git a/pkg/kv/kvserver/replica_write.go b/pkg/kv/kvserver/replica_write.go index b9d74ea8bafd..51c0c333a7b6 100644 --- a/pkg/kv/kvserver/replica_write.go +++ b/pkg/kv/kvserver/replica_write.go @@ -101,7 +101,6 @@ func (r *Replica) executeWriteBatch( readOnlyCmdMu.RLock() defer func() { if readOnlyCmdMu != nil { - // nolint:deferunlock readOnlyCmdMu.RUnlock() } }() diff --git a/pkg/kv/kvserver/scanner.go b/pkg/kv/kvserver/scanner.go index 030ac26cbf75..05f6334db832 100644 --- a/pkg/kv/kvserver/scanner.go +++ b/pkg/kv/kvserver/scanner.go @@ -319,7 +319,6 @@ func (rs *replicaScanner) scanLoop() { func (rs *replicaScanner) waitEnabled() bool { rs.mu.Lock() rs.mu.waitEnabledCount++ - // nolint:deferunlock rs.mu.Unlock() for { if !rs.GetDisabled() { diff --git a/pkg/kv/kvserver/scanner_test.go b/pkg/kv/kvserver/scanner_test.go index f33efcb9581a..511206c76d2f 100644 --- a/pkg/kv/kvserver/scanner_test.go +++ b/pkg/kv/kvserver/scanner_test.go @@ -74,7 +74,6 @@ func (rs *testRangeSet) Visit(visitor func(*Replica) bool) { rs.visited = 0 rs.replicasByKey.Ascend(func(i btree.Item) bool { rs.visited++ - // nolint:deferunlock rs.Unlock() defer rs.Lock() return visitor((*Replica)(i.(*btreeReplica))) @@ -124,7 +123,6 @@ func (tq *testQueue) Start(stopper *stop.Stopper) { done := func() { tq.Lock() tq.done = true - // nolint:deferunlock tq.Unlock() } @@ -137,7 +135,6 @@ func (tq *testQueue) Start(stopper *stop.Stopper) { tq.ranges = tq.ranges[1:] tq.processed++ } - // nolint:deferunlock tq.Unlock() case <-stopper.ShouldQuiesce(): done() diff --git a/pkg/kv/kvserver/scheduler.go b/pkg/kv/kvserver/scheduler.go index 628266d0d796..06bc40b91d2d 100644 --- a/pkg/kv/kvserver/scheduler.go +++ b/pkg/kv/kvserver/scheduler.go @@ -291,7 +291,6 @@ func (s *raftScheduler) Start(stopper *stop.Stopper) { for _, shard := range s.shards { shard.Lock() shard.stopped = true - // nolint:deferunlock shard.Unlock() shard.cond.Broadcast() } @@ -369,7 +368,6 @@ func (ss *raftSchedulerShard) worker( var id roachpb.RangeID for { if ss.stopped { - // nolint:deferunlock ss.Unlock() return } @@ -495,7 +493,6 @@ func (s *raftScheduler) enqueue1(addFlags raftScheduleFlags, id roachpb.RangeID) shard := s.shards[shardIdx] shard.Lock() n := shard.enqueue1Locked(addFlags, id, now) - // nolint:deferunlock shard.Unlock() shard.signal(n) } @@ -515,13 +512,11 @@ func (ss *raftSchedulerShard) enqueueN(addFlags raftScheduleFlags, ids ...roachp for i, id := range ids { count += ss.enqueue1Locked(addFlags, id, now) if (i+1)%enqueueChunkSize == 0 { - // nolint:deferunlock ss.Unlock() now = nowNanos() ss.Lock() } } - // nolint:deferunlock ss.Unlock() return count } diff --git a/pkg/kv/kvserver/scheduler_test.go b/pkg/kv/kvserver/scheduler_test.go index 2123b6cbe6f4..473ac8186d49 100644 --- a/pkg/kv/kvserver/scheduler_test.go +++ b/pkg/kv/kvserver/scheduler_test.go @@ -151,7 +151,6 @@ func newTestProcessor() *testProcessor { func (p *testProcessor) onReady(f func(roachpb.RangeID)) { p.mu.Lock() p.mu.ready = f - // nolint:deferunlock p.mu.Unlock() } @@ -160,7 +159,6 @@ func (p *testProcessor) processReady(rangeID roachpb.RangeID) { p.mu.raftReady[rangeID]++ onReady := p.mu.ready p.mu.ready = nil - // nolint:deferunlock p.mu.Unlock() if onReady != nil { onReady(rangeID) @@ -170,7 +168,6 @@ func (p *testProcessor) processReady(rangeID roachpb.RangeID) { func (p *testProcessor) processRequestQueue(_ context.Context, rangeID roachpb.RangeID) bool { p.mu.Lock() p.mu.raftRequest[rangeID]++ - // nolint:deferunlock p.mu.Unlock() return false } @@ -178,7 +175,6 @@ func (p *testProcessor) processRequestQueue(_ context.Context, rangeID roachpb.R func (p *testProcessor) processTick(_ context.Context, rangeID roachpb.RangeID) bool { p.mu.Lock() p.mu.raftTick[rangeID]++ - // nolint:deferunlock p.mu.Unlock() return false } diff --git a/pkg/kv/kvserver/spanlatch/manager.go b/pkg/kv/kvserver/spanlatch/manager.go index d08b1b5c7571..32fe84e42081 100644 --- a/pkg/kv/kvserver/spanlatch/manager.go +++ b/pkg/kv/kvserver/spanlatch/manager.go @@ -264,7 +264,6 @@ func (m *Manager) WaitFor(ctx context.Context, spans *spanset.SpanSet, pp poison m.mu.Lock() snap := m.snapshotLocked(spans) defer snap.close() - // nolint:deferunlock m.mu.Unlock() return m.wait(ctx, lg, snap) @@ -362,7 +361,6 @@ func (m *Manager) sequence(spans *spanset.SpanSet, pp poison.Policy) (*Guard, sn m.mu.Lock() snap := m.snapshotLocked(spans) m.insertLocked(lg) - // nolint:deferunlock m.mu.Unlock() return lg, snap } @@ -613,7 +611,6 @@ func (m *Manager) Release(lg *Guard) { m.mu.Lock() m.removeLocked(lg) - // nolint:deferunlock m.mu.Unlock() } diff --git a/pkg/kv/kvserver/split_delay_helper.go b/pkg/kv/kvserver/split_delay_helper.go index bd79d0d30212..6f39144191de 100644 --- a/pkg/kv/kvserver/split_delay_helper.go +++ b/pkg/kv/kvserver/split_delay_helper.go @@ -44,7 +44,6 @@ func (sdh *splitDelayHelper) RaftStatus(ctx context.Context) (roachpb.RangeID, * }, ) } - // nolint:deferunlock r.mu.RUnlock() return r.RangeID, raftStatus } diff --git a/pkg/kv/kvserver/split_queue_test.go b/pkg/kv/kvserver/split_queue_test.go index a819fdb3b7a5..60b891749505 100644 --- a/pkg/kv/kvserver/split_queue_test.go +++ b/pkg/kv/kvserver/split_queue_test.go @@ -107,7 +107,6 @@ func TestSplitQueueShouldQueue(t *testing.T) { repl.mu.Lock() repl.mu.state.Stats = &enginepb.MVCCStats{KeyBytes: test.bytes} - // nolint:deferunlock repl.mu.Unlock() conf := roachpb.TestingDefaultSpanConfig() conf.RangeMaxBytes = test.maxBytes diff --git a/pkg/kv/kvserver/split_trigger_helper.go b/pkg/kv/kvserver/split_trigger_helper.go index ad023f607386..a9310ded8803 100644 --- a/pkg/kv/kvserver/split_trigger_helper.go +++ b/pkg/kv/kvserver/split_trigger_helper.go @@ -30,7 +30,6 @@ func (rd *replicaMsgAppDropper) Args() (initialized bool, age time.Duration) { r.mu.RLock() initialized = r.IsInitialized() creationTime := r.creationTime - // nolint:deferunlock r.mu.RUnlock() age = timeutil.Since(creationTime) return initialized, age diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index 2fa2f034ff19..317eb5aac387 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -501,7 +501,6 @@ func (rs *storeReplicaVisitor) Visit(visitor func(*Replica) bool) { repl.mu.RLock() destroyed := repl.mu.destroyStatus initialized := repl.IsInitialized() - // nolint:deferunlock repl.mu.RUnlock() if initialized && destroyed.IsAlive() && !visitor(repl) { break @@ -1430,7 +1429,6 @@ func NewStore( s.coalescedMu.Lock() s.coalescedMu.heartbeats = map[roachpb.StoreIdent][]kvserverpb.RaftHeartbeat{} s.coalescedMu.heartbeatResponses = map[roachpb.StoreIdent][]kvserverpb.RaftHeartbeat{} - // nolint:deferunlock s.coalescedMu.Unlock() s.mu.Lock() @@ -1438,17 +1436,14 @@ func NewStore( s.mu.replicasByKey = newStoreReplicaBTree() s.mu.creatingReplicas = map[roachpb.RangeID]struct{}{} s.mu.uninitReplicas = map[roachpb.RangeID]*Replica{} - // nolint:deferunlock s.mu.Unlock() s.unquiescedReplicas.Lock() s.unquiescedReplicas.m = map[roachpb.RangeID]struct{}{} - // nolint:deferunlock s.unquiescedReplicas.Unlock() s.rangefeedReplicas.Lock() s.rangefeedReplicas.m = map[roachpb.RangeID]struct{}{} - // nolint:deferunlock s.rangefeedReplicas.Unlock() s.tsCache = tscache.New(cfg.Clock) @@ -1745,7 +1740,6 @@ func (s *Store) SetDraining(drain bool, reporter func(int, redact.SafeString), v if nextLease != (roachpb.Lease{}) && nextLease.OwnedBy(s.StoreID()) { llHandle = r.mu.pendingLeaseRequest.JoinRequest() } - // nolint:deferunlock r.mu.Unlock() if llHandle != nil { @@ -2075,7 +2069,6 @@ func (s *Store) Start(ctx context.Context, stopper *stop.Stopper) error { if err == nil { err = s.addToReplicasByKeyLocked(rep, rep.Desc()) } - // nolint:deferunlock s.mu.Unlock() if err != nil { return err @@ -2265,7 +2258,6 @@ func (s *Store) startRangefeedUpdater(ctx context.Context) { for rangeID := range s.rangefeedReplicas.m { rangeIDs = append(rangeIDs, rangeID) } - // nolint:deferunlock s.rangefeedReplicas.Unlock() // Sort the range IDs so that we notify them in the same order on each // iteration. With the pacing below, this helps to ensure a consistent @@ -2364,14 +2356,12 @@ func (s *Store) startRangefeedUpdater(ctx context.Context) { func (s *Store) addReplicaWithRangefeed(rangeID roachpb.RangeID) { s.rangefeedReplicas.Lock() s.rangefeedReplicas.m[rangeID] = struct{}{} - // nolint:deferunlock s.rangefeedReplicas.Unlock() } func (s *Store) removeReplicaWithRangefeed(rangeID roachpb.RangeID) { s.rangefeedReplicas.Lock() delete(s.rangefeedReplicas.m, rangeID) - // nolint:deferunlock s.rangefeedReplicas.Unlock() } @@ -2857,7 +2847,6 @@ func (s *Store) Capacity(ctx context.Context, useCached bool) (roachpb.StoreCapa { s.ioThreshold.Lock() capacity.IOThreshold = *s.ioThreshold.t - // nolint:deferunlock s.ioThreshold.Unlock() } capacity.BytesPerReplica = roachpb.PercentilesFromData(bytesPerReplica) @@ -3002,14 +2991,12 @@ func (s *Store) updateReplicationGauges(ctx context.Context) error { s.mu.RLock() uninitializedCount = int64(len(s.mu.uninitReplicas)) - // nolint:deferunlock s.mu.RUnlock() // TODO(kaisun314,kvoli): move this to a per-store admission control metrics // struct when available. See pkg/util/admission/granter.go. s.ioThreshold.Lock() ioOverload, _ = s.ioThreshold.t.Score() - // nolint:deferunlock s.ioThreshold.Unlock() // We want to avoid having to read this multiple times during the replica @@ -3191,7 +3178,6 @@ func (s *Store) checkpointSpans(desc *roachpb.RangeDescriptor) []roachpb.Span { s.mu.RLock() left := s.mu.replicasByKey.LookupPrecedingReplica(context.Background(), desc.StartKey) right := s.mu.replicasByKey.LookupNextReplica(context.Background(), desc.EndKey) - // nolint:deferunlock s.mu.RUnlock() // Cover all range IDs (prevID, desc.RangeID, nextID) using a continuous span. @@ -3747,7 +3733,6 @@ func (s *storeForTruncatorImpl) acquireReplicaForTruncator( defer r.mu.Unlock() return r.mu.destroyStatus.IsAlive() }(); !isAlive { - // nolint:deferunlock r.raftMu.Unlock() return nil } diff --git a/pkg/kv/kvserver/store_create_replica.go b/pkg/kv/kvserver/store_create_replica.go index 21f5d34c8824..ee78491aed59 100644 --- a/pkg/kv/kvserver/store_create_replica.go +++ b/pkg/kv/kvserver/store_create_replica.go @@ -112,9 +112,7 @@ func (s *Store) tryGetReplica( // The current replica is removed, go back around. if repl.mu.destroyStatus.Removed() { - // nolint:deferunlock repl.mu.RUnlock() - // nolint:deferunlock repl.raftMu.Unlock() return nil, errRetry } @@ -185,17 +183,14 @@ func (s *Store) tryGetOrCreateReplica( if _, ok := s.mu.creatingReplicas[rangeID]; ok { // Lost the race - another goroutine is currently creating that replica. Let // the caller retry so that they can eventually see it. - // nolint:deferunlock s.mu.Unlock() return nil, false, errRetry } s.mu.creatingReplicas[rangeID] = struct{}{} - // nolint:deferunlock s.mu.Unlock() defer func() { s.mu.Lock() delete(s.mu.creatingReplicas, rangeID) - // nolint:deferunlock s.mu.Unlock() }() // Now we are the only goroutine trying to create a replica for this rangeID. @@ -235,9 +230,7 @@ func (s *Store) tryGetOrCreateReplica( // snapshot is applied. // TODO(pavelkalinnikov): make this branch error-less. if err := s.addToReplicasByRangeIDLocked(repl); err != nil { - // nolint:deferunlock s.mu.Unlock() - // nolint:deferunlock repl.raftMu.Unlock() return nil, false, err } diff --git a/pkg/kv/kvserver/store_gossip.go b/pkg/kv/kvserver/store_gossip.go index cf6c4a18406c..4d33e385f5ec 100644 --- a/pkg/kv/kvserver/store_gossip.go +++ b/pkg/kv/kvserver/store_gossip.go @@ -364,7 +364,6 @@ func (s *StoreGossip) GossipStore(ctx context.Context, useCached bool) error { // was required. s.cachedCapacity.Lock() s.cachedCapacity.lastGossiped = storeDesc.Capacity - // nolint:deferunlock s.cachedCapacity.Unlock() // Unique gossip key per store. @@ -425,7 +424,6 @@ func (s *StoreGossip) RecordNewPerSecondStats(newQPS, newWPS float64) { s.cachedCapacity.Lock() s.cachedCapacity.cached.QueriesPerSecond = newQPS s.cachedCapacity.cached.WritesPerSecond = newWPS - // nolint:deferunlock s.cachedCapacity.Unlock() if shouldGossip, reason := s.shouldGossipOnCapacityDelta(); shouldGossip { diff --git a/pkg/kv/kvserver/store_pool_test.go b/pkg/kv/kvserver/store_pool_test.go index 9c0e1c4e5f30..78baa02a3949 100644 --- a/pkg/kv/kvserver/store_pool_test.go +++ b/pkg/kv/kvserver/store_pool_test.go @@ -112,7 +112,6 @@ func TestStorePoolUpdateLocalStore(t *testing.T) { KeyBytes: 2, ValBytes: 4, } - // nolint:deferunlock replica.mu.Unlock() replica.loadStats = load.NewReplicaLoad(clock, nil) for _, store := range stores { diff --git a/pkg/kv/kvserver/store_raft.go b/pkg/kv/kvserver/store_raft.go index 51068cdfeed2..90d09b0dddc6 100644 --- a/pkg/kv/kvserver/store_raft.go +++ b/pkg/kv/kvserver/store_raft.go @@ -560,7 +560,6 @@ func (s *Store) HandleRaftResponse( !repl.mu.destroyStatus.IsAlive() || // Ignore if we want to test the replicaGC queue. s.TestingKnobs().DisableEagerReplicaRemoval { - // nolint:deferunlock repl.mu.Unlock() return nil } @@ -747,7 +746,6 @@ func (s *Store) nodeIsLiveCallback(l livenesspb.Liveness) { r.mu.RLock() quiescent := r.mu.quiescent lagging := r.mu.laggingFollowersOnQuiesce - // nolint:deferunlock r.mu.RUnlock() if quiescent && lagging.MemberStale(l) { r.maybeUnquiesce(false /* wakeLeader */, false /* mayCampaign */) // already leader @@ -784,7 +782,6 @@ func (s *Store) processRaft(ctx context.Context) { makeProposalResultErr( kvpb.NewAmbiguousResultErrorf("store is stopping"))) } - // nolint:deferunlock r.mu.Unlock() return true }) @@ -814,7 +811,6 @@ func (s *Store) raftTickLoop(ctx context.Context) { for rangeID := range s.unquiescedReplicas.m { batch.Add(rangeID) } - // nolint:deferunlock s.unquiescedReplicas.Unlock() s.scheduler.EnqueueRaftTicks(batch) @@ -948,7 +944,6 @@ func (s *Store) sendQueuedHeartbeats(ctx context.Context) { heartbeatResponses := s.coalescedMu.heartbeatResponses s.coalescedMu.heartbeats = map[roachpb.StoreIdent][]kvserverpb.RaftHeartbeat{} s.coalescedMu.heartbeatResponses = map[roachpb.StoreIdent][]kvserverpb.RaftHeartbeat{} - // nolint:deferunlock s.coalescedMu.Unlock() var beatsSent int diff --git a/pkg/kv/kvserver/store_remove_replica.go b/pkg/kv/kvserver/store_remove_replica.go index 02d0b51f12f1..75b5951d4710 100644 --- a/pkg/kv/kvserver/store_remove_replica.go +++ b/pkg/kv/kvserver/store_remove_replica.go @@ -161,7 +161,6 @@ func (s *Store) removeInitializedReplicaRaftMuLocked( // This is a fatal error because uninitialized replicas shouldn't make it // this far. This method will need some changes when we introduce GC of // uninitialized replicas. - // nolint:deferunlock s.mu.Unlock() log.Fatalf(ctx, "replica %+v unexpectedly overlapped by %+v", rep, it.item) } @@ -253,9 +252,7 @@ func (s *Store) removeUninitializedReplicaRaftMuLocked( // because we should have already checked this under the raftMu // before calling this method. if rep.mu.destroyStatus.Removed() { - // nolint:deferunlock rep.mu.Unlock() - // nolint:deferunlock rep.readOnlyCmdMu.Unlock() log.Fatalf(ctx, "uninitialized replica unexpectedly already removed") } @@ -307,7 +304,6 @@ func (s *Store) unlinkReplicaByRangeIDLocked(ctx context.Context, rangeID roachp s.mu.AssertHeld() s.unquiescedReplicas.Lock() delete(s.unquiescedReplicas.m, rangeID) - // nolint:deferunlock s.unquiescedReplicas.Unlock() delete(s.mu.uninitReplicas, rangeID) s.mu.replicasByRangeID.Delete(rangeID) diff --git a/pkg/kv/kvserver/store_snapshot.go b/pkg/kv/kvserver/store_snapshot.go index 304e213c8d88..e93bbe5e6b4b 100644 --- a/pkg/kv/kvserver/store_snapshot.go +++ b/pkg/kv/kvserver/store_snapshot.go @@ -1200,7 +1200,6 @@ func (s *Store) canAcceptSnapshotLocked( existingDesc := existingRepl.mu.state.Desc existingIsInitialized := existingDesc.IsInitialized() existingDestroyStatus := existingRepl.mu.destroyStatus - // nolint:deferunlock existingRepl.mu.RUnlock() if existingIsInitialized { diff --git a/pkg/kv/kvserver/store_split.go b/pkg/kv/kvserver/store_split.go index 1f6d9803ef30..6dcd81e16528 100644 --- a/pkg/kv/kvserver/store_split.go +++ b/pkg/kv/kvserver/store_split.go @@ -229,7 +229,6 @@ func prepareRightReplicaForSplit( r.mu.RLock() minLeaseProposedTS := r.mu.minLeaseProposedTS minValidObservedTS := r.mu.minValidObservedTimestamp - // nolint:deferunlock r.mu.RUnlock() // If the RHS replica of the split is not removed, then it has been obtained diff --git a/pkg/kv/kvserver/store_test.go b/pkg/kv/kvserver/store_test.go index 2bcc522d3c3b..2b8c7c4a05c4 100644 --- a/pkg/kv/kvserver/store_test.go +++ b/pkg/kv/kvserver/store_test.go @@ -697,7 +697,6 @@ func TestReplicasByKey(t *testing.T) { desc := *rep.mu.state.Desc // shallow copy to replace desc wholesale desc.EndKey = roachpb.RKey("e") rep.mu.state.Desc = &desc - // nolint:deferunlock rep.mu.Unlock() // Ensure that this shrinkage is recognized by future additions to replicasByKey. @@ -768,7 +767,6 @@ func TestStoreRemoveReplicaDestroy(t *testing.T) { repl1.mu.RLock() expErr := repl1.mu.destroyStatus.err - // nolint:deferunlock repl1.mu.RUnlock() if expErr == nil { @@ -2781,7 +2779,6 @@ func TestStoreGCThreshold(t *testing.T) { repl.mu.Lock() gcThreshold := *repl.mu.state.GCThreshold pgcThreshold, err := repl.mu.stateLoader.LoadGCThreshold(context.Background(), store.TODOEngine()) - // nolint:deferunlock repl.mu.Unlock() if err != nil { t.Fatal(err) @@ -2858,7 +2855,6 @@ func TestStoreRangePlaceholders(t *testing.T) { s.mu.Lock() numPlaceholders := len(s.mu.replicaPlaceholders) - // nolint:deferunlock s.mu.Unlock() if numPlaceholders != 0 { @@ -3044,7 +3040,6 @@ func TestStoreRemovePlaceholderOnRaftIgnored(t *testing.T) { { s.mu.Lock() err := s.addPlaceholderLocked(placeholder) - // nolint:deferunlock s.mu.Unlock() require.NoError(t, err) } @@ -3061,7 +3056,6 @@ func TestStoreRemovePlaceholderOnRaftIgnored(t *testing.T) { testutils.SucceedsSoon(t, func() error { s.mu.Lock() numPlaceholders := len(s.mu.replicaPlaceholders) - // nolint:deferunlock s.mu.Unlock() if numPlaceholders != 0 { @@ -3381,7 +3375,6 @@ func TestReserveSnapshotFullnessLimit(t *testing.T) { s.cfg.StorePool.DetailsMu.Lock() s.cfg.StorePool.GetStoreDetailLocked(desc.StoreID).Desc = desc - // nolint:deferunlock s.cfg.StorePool.DetailsMu.Unlock() if n := s.ReservationCount(); n != 0 { @@ -3406,7 +3399,6 @@ func TestReserveSnapshotFullnessLimit(t *testing.T) { desc.Capacity.Used = desc.Capacity.Capacity - desc.Capacity.Available s.cfg.StorePool.DetailsMu.Lock() s.cfg.StorePool.GetStoreDetailLocked(desc.StoreID).Desc = desc - // nolint:deferunlock s.cfg.StorePool.DetailsMu.Unlock() if n := s.ReservationCount(); n != 0 { diff --git a/pkg/kv/kvserver/stores_server.go b/pkg/kv/kvserver/stores_server.go index 12351f495550..b4040707b7bc 100644 --- a/pkg/kv/kvserver/stores_server.go +++ b/pkg/kv/kvserver/stores_server.go @@ -91,7 +91,6 @@ func (is Server) WaitForApplication( } repl.mu.RLock() leaseAppliedIndex := repl.mu.state.LeaseAppliedIndex - // nolint:deferunlock repl.mu.RUnlock() if leaseAppliedIndex >= req.LeaseIndex { // For performance reasons, we don't sync to disk when diff --git a/pkg/kv/kvserver/ts_maintenance_queue_test.go b/pkg/kv/kvserver/ts_maintenance_queue_test.go index a4eabe2ed7a0..96a95aaaeee6 100644 --- a/pkg/kv/kvserver/ts_maintenance_queue_test.go +++ b/pkg/kv/kvserver/ts_maintenance_queue_test.go @@ -171,7 +171,6 @@ func TestTimeSeriesMaintenanceQueue(t *testing.T) { if a, e := model.pruneSeenEndKeys, expectedEndKeys; !reflect.DeepEqual(a, e) { t.Errorf("end keys seen by MaintainTimeSeries did not match expectation: %s", pretty.Diff(a, e)) } - // nolint:deferunlock model.Unlock() testutils.SucceedsSoon(t, func() error { @@ -200,7 +199,6 @@ func TestTimeSeriesMaintenanceQueue(t *testing.T) { if a, e := model.pruneCalled, len(expectedStartKeys); a != e { t.Errorf("MaintainTimeSeries called %d times; expected %d", a, e) } - // nolint:deferunlock model.Unlock() // Move clock forward and force to scan again. diff --git a/pkg/kv/kvserver/txnwait/queue.go b/pkg/kv/kvserver/txnwait/queue.go index 4eec450d9775..d3f19f97c63a 100644 --- a/pkg/kv/kvserver/txnwait/queue.go +++ b/pkg/kv/kvserver/txnwait/queue.go @@ -225,7 +225,6 @@ func (pt *pendingTxn) getDependentsSet() map[uuid.UUID]struct{} { for txnID := range push.mu.dependents { set[txnID] = struct{}{} } - // nolint:deferunlock push.mu.Unlock() } } @@ -403,7 +402,6 @@ func (q *Queue) UpdateTxn(ctx context.Context, txn *roachpb.Transaction) { if q.mu.txns == nil { // Not enabled; do nothing. - // nolint:deferunlock q.mu.Unlock() return } @@ -500,7 +498,6 @@ func (q *Queue) MaybeWaitForPush( // ContainsKey check is done under the txn wait queue's lock to // ensure that it's not cleared before an incorrect insertion happens. if q.mu.txns == nil || !q.RangeContainsKeyLocked(req.Key) { - // nolint:deferunlock q.mu.Unlock() return nil, nil } @@ -542,7 +539,6 @@ func (q *Queue) MaybeWaitForPush( defer func() { q.mu.Lock() pending.waitingPushes.Remove(pushElem) - // nolint:deferunlock q.mu.Unlock() }() @@ -752,7 +748,6 @@ func (q *Queue) waitForPush( // so that they continue with a query of new dependents added here. q.mu.Lock() q.releaseWaitingQueriesLocked(ctx, req.PusheeTxn.ID) - // nolint:deferunlock q.mu.Unlock() if haveDependency { @@ -797,7 +792,6 @@ func (q *Queue) MaybeWaitForQuery(ctx context.Context, req *kvpb.QueryTxnRequest // ContainsKey check is done under the txn wait queue's lock to // ensure that it's not cleared before an incorrect insertion happens. if q.mu.txns == nil || !q.RangeContainsKeyLocked(req.Key) { - // nolint:deferunlock q.mu.Unlock() return nil } @@ -842,7 +836,6 @@ func (q *Queue) MaybeWaitForQuery(ctx context.Context, req *kvpb.QueryTxnRequest if query.count == 0 && query == q.mu.queries[req.Txn.ID] { delete(q.mu.queries, req.Txn.ID) } - // nolint:deferunlock q.mu.Unlock() }() @@ -892,7 +885,6 @@ func (q *Queue) startQueryPusherTxn( } } pusher := push.req.PusherTxn.Clone() - // nolint:deferunlock push.mu.Unlock() if err := q.cfg.Stopper.RunAsyncTask( @@ -931,7 +923,6 @@ func (q *Queue) startQueryPusherTxn( for _, txnID := range waitingTxns { push.mu.dependents[txnID] = struct{}{} } - // nolint:deferunlock push.mu.Unlock() // Send an update of the pusher txn. @@ -1044,7 +1035,6 @@ func (q *Queue) TrackedTxns() map[uuid.UUID]struct{} { for k := range q.mu.txns { m[k] = struct{}{} } - // nolint:deferunlock q.mu.RUnlock() return m } diff --git a/pkg/kv/txn.go b/pkg/kv/txn.go index 8b491dfde8b9..84900baf5eb4 100644 --- a/pkg/kv/txn.go +++ b/pkg/kv/txn.go @@ -333,7 +333,6 @@ func (txn *Txn) TestingSetPriority(priority enginepb.TxnPriority) { if err := txn.mu.sender.SetUserPriority(txn.mu.userPriority); err != nil { log.Fatalf(context.TODO(), "%+v", err) } - // nolint:deferunlock txn.mu.Unlock() } @@ -1162,7 +1161,6 @@ func (txn *Txn) Send( txn.mu.Lock() requestTxnID := txn.mu.ID sender := txn.mu.sender - // nolint:deferunlock txn.mu.Unlock() br, pErr := txn.db.sendUsingSender(ctx, ba, sender) diff --git a/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/authorizer.go b/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/authorizer.go index 946c3d0c9a14..b2ba1caddd34 100644 --- a/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/authorizer.go +++ b/pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer/authorizer.go @@ -369,7 +369,6 @@ func (a *Authorizer) getMode( if selectedMode == authorizerModeOn { a.Lock() reader := a.capabilitiesReader - // nolint:deferunlock a.Unlock() if reader == nil { // The server has started but the reader hasn't started/bound diff --git a/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/watcher.go b/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/watcher.go index b9bed04266b9..7e987af8d809 100644 --- a/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/watcher.go +++ b/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/watcher.go @@ -112,7 +112,6 @@ func New( func (w *Watcher) getInternal(tenantID roachpb.TenantID) *watcherEntry { w.mu.RLock() cp, found := w.mu.store[tenantID] - // nolint:deferunlock w.mu.RUnlock() if found { return cp diff --git a/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/watcher_test.go b/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/watcher_test.go index 5819ae9da0c6..bfae070171d9 100644 --- a/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/watcher_test.go +++ b/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/watcher_test.go @@ -134,7 +134,6 @@ func TestDataDriven(t *testing.T) { PreExit: func() { mu.Lock() mu.rangeFeedRunning = false - // nolint:deferunlock mu.Unlock() // Block until the test directives indicate otherwise. <-restartAfterErrCh @@ -188,7 +187,6 @@ func TestDataDriven(t *testing.T) { receivedUpdates := mu.receivedUpdates mu.receivedUpdates = mu.receivedUpdates[:0] // clear out buffer updateType := mu.receivedUpdateType - // nolint:deferunlock mu.Unlock() // De-duplicate updates. We want a stable sort here because the diff --git a/pkg/obs/event_exporter.go b/pkg/obs/event_exporter.go index f780989ef408..0d9fece6b47b 100644 --- a/pkg/obs/event_exporter.go +++ b/pkg/obs/event_exporter.go @@ -337,7 +337,6 @@ func (s *EventsExporter) Start(ctx context.Context, stopper *stop.Stopper) error _, err := s.otelClient.Export(ctx, req, grpc.WaitForReady(true)) s.buf.mu.Lock() s.buf.mu.memAccount.Shrink(ctx, int64(msgSize)) - // nolint:deferunlock s.buf.mu.Unlock() if err != nil { log.Warningf(ctx, "failed to export events: %s", err) diff --git a/pkg/roachprod/install/cluster_synced.go b/pkg/roachprod/install/cluster_synced.go index ba4dc094206e..bee99c3f8703 100644 --- a/pkg/roachprod/install/cluster_synced.go +++ b/pkg/roachprod/install/cluster_synced.go @@ -1937,7 +1937,6 @@ func (c *SyncedCluster) Put( } else { lines[r.index] = "done" } - // nolint:deferunlock linesMu.Unlock() } } @@ -1952,7 +1951,6 @@ func (c *SyncedCluster) Put( } fmt.Fprintf(&writer, "\n") } - // nolint:deferunlock linesMu.Unlock() _ = writer.Flush(l.Stdout) spinnerIdx++ @@ -1965,7 +1963,6 @@ func (c *SyncedCluster) Put( for i := range lines { l.Printf(" %2d: %s", nodes[i], lines[i]) } - // nolint:deferunlock linesMu.Unlock() } @@ -2311,7 +2308,6 @@ func (c *SyncedCluster) Get( } else { lines[r.index] = "done" } - // nolint:deferunlock linesMu.Unlock() } } @@ -2326,7 +2322,6 @@ func (c *SyncedCluster) Get( } fmt.Fprintf(&writer, "\n") } - // nolint:deferunlock linesMu.Unlock() _ = writer.Flush(l.Stdout) spinnerIdx++ @@ -2338,7 +2333,6 @@ func (c *SyncedCluster) Get( for i := range lines { l.Printf(" %2d: %s", nodes[i], lines[i]) } - // nolint:deferunlock linesMu.Unlock() } diff --git a/pkg/roachprod/roachprod.go b/pkg/roachprod/roachprod.go index 44bf83dabec4..0e8998807af0 100644 --- a/pkg/roachprod/roachprod.go +++ b/pkg/roachprod/roachprod.go @@ -1142,7 +1142,6 @@ func Pprof(ctx context.Context, l *logger.Logger, clusterName string, opts Pprof mu.Lock() outputFiles = append(outputFiles, outputFile) - // nolint:deferunlock mu.Unlock() return res, nil }, install.WithDisplay(description)) @@ -1736,7 +1735,6 @@ func CreateSnapshot( volumeSnapshot.Name, volumeSnapshot.ID, volume.Name, volume.ProviderResourceID, node) volumesSnapshotMu.Lock() volumesSnapshotMu.snapshots = append(volumesSnapshotMu.snapshots, volumeSnapshot) - // nolint:deferunlock volumesSnapshotMu.Unlock() } return nil diff --git a/pkg/roachprod/vm/aws/aws.go b/pkg/roachprod/vm/aws/aws.go index 9747d501b359..04434cd91e56 100644 --- a/pkg/roachprod/vm/aws/aws.go +++ b/pkg/roachprod/vm/aws/aws.go @@ -748,7 +748,6 @@ func (p *Provider) listRegions( } mux.Lock() ret = append(ret, vms...) - // nolint:deferunlock mux.Unlock() return nil }) diff --git a/pkg/roachprod/vm/azure/auth.go b/pkg/roachprod/vm/azure/auth.go index 5099706963f0..d0f988420e04 100644 --- a/pkg/roachprod/vm/azure/auth.go +++ b/pkg/roachprod/vm/azure/auth.go @@ -28,7 +28,6 @@ import ( func (p *Provider) getAuthorizer() (ret autorest.Authorizer, err error) { p.mu.Lock() ret = p.mu.authorizer - // nolint:deferunlock p.mu.Unlock() if ret != nil { return @@ -40,7 +39,6 @@ func (p *Provider) getAuthorizer() (ret autorest.Authorizer, err error) { if err == nil { p.mu.Lock() p.mu.authorizer = ret - // nolint:deferunlock p.mu.Unlock() } else { err = errors.Wrap(err, "could got get Azure auth token") diff --git a/pkg/roachprod/vm/azure/azure.go b/pkg/roachprod/vm/azure/azure.go index ef5f6e57202d..ebb8172ae28a 100644 --- a/pkg/roachprod/vm/azure/azure.go +++ b/pkg/roachprod/vm/azure/azure.go @@ -248,7 +248,6 @@ func (p *Provider) Create( p.mu.Lock() subnet, ok := p.mu.subnets[location] - // nolint:deferunlock p.mu.Unlock() if !ok { return errors.Errorf("missing subnet for location %q", location) @@ -816,7 +815,6 @@ func (p *Provider) createNIC( p.mu.Lock() sg := p.mu.securityGroups[p.getVnetNetworkSecurityGroupName(*group.Location)] - // nolint:deferunlock p.mu.Unlock() future, err := client.CreateOrUpdate(ctx, *group.Name, *ip.Name, network.Interface{ @@ -856,7 +854,6 @@ func (p *Provider) getOrCreateNetworkSecurityGroup( ) (network.SecurityGroup, error) { p.mu.Lock() group, ok := p.mu.securityGroups[name] - // nolint:deferunlock p.mu.Unlock() if ok { return group, nil @@ -877,7 +874,6 @@ func (p *Provider) getOrCreateNetworkSecurityGroup( cacheAndReturn := func(group network.SecurityGroup) (network.SecurityGroup, error) { p.mu.Lock() p.mu.securityGroups[name] = group - // nolint:deferunlock p.mu.Unlock() return group, nil } @@ -1102,7 +1098,6 @@ func (p *Provider) createVNets( for _, location := range providerOpts.Locations { p.mu.Lock() group := p.mu.resourceGroups[vnetResourceGroupName(location)] - // nolint:deferunlock p.mu.Unlock() // Prefix already exists for the resource group. if prefixString := group.Tags[tagSubnet]; prefixString != nil { @@ -1120,7 +1115,6 @@ func (p *Provider) createVNets( prefixesByLocation[location] = prefix p.mu.Lock() group := p.mu.resourceGroups[vnetResourceGroupName(location)] - // nolint:deferunlock p.mu.Unlock() group, err = setVNetSubnetPrefix(group, prefix) if err != nil { @@ -1130,7 +1124,6 @@ func (p *Provider) createVNets( // the cached entry to reflect that. p.mu.Lock() p.mu.resourceGroups[vnetResourceGroupName(location)] = group - // nolint:deferunlock p.mu.Unlock() } } @@ -1145,7 +1138,6 @@ func (p *Provider) createVNets( p.mu.Lock() resourceGroup := p.mu.resourceGroups[vnetResourceGroupName(location)] networkSecurityGroup := p.mu.securityGroups[p.getVnetNetworkSecurityGroupName(location)] - // nolint:deferunlock p.mu.Unlock() if vnet, _, err := p.createVNet(l, ctx, resourceGroup, networkSecurityGroup, prefix, providerOpts); err == nil { ret[location] = vnet @@ -1218,7 +1210,6 @@ func (p *Provider) createVNet( subnet = (*vnet.Subnets)[0] p.mu.Lock() p.mu.subnets[*resourceGroup.Location] = subnet - // nolint:deferunlock p.mu.Unlock() l.Printf("created Azure VNet %q in %q with prefix %d", vnetName, *resourceGroup.Name, prefix) return @@ -1393,7 +1384,6 @@ func (p *Provider) getOrCreateResourceGroup( // First, check the local provider cache. p.mu.Lock() group, ok := p.mu.resourceGroups[name] - // nolint:deferunlock p.mu.Unlock() if ok { return group, nil @@ -1402,7 +1392,6 @@ func (p *Provider) getOrCreateResourceGroup( cacheAndReturn := func(group resources.Group) (resources.Group, error) { p.mu.Lock() p.mu.resourceGroups[name] = group - // nolint:deferunlock p.mu.Unlock() return group, nil } @@ -1494,7 +1483,6 @@ func (p *Provider) getSubscription( ) (sub subscriptions.Subscription, err error) { p.mu.Lock() sub = p.mu.subscription - // nolint:deferunlock p.mu.Unlock() if sub.SubscriptionID != nil { @@ -1516,7 +1504,6 @@ func (p *Provider) getSubscription( p.mu.Lock() p.mu.subscription = page.Values()[0] - // nolint:deferunlock p.mu.Unlock() } return diff --git a/pkg/rpc/clock_offset_test.go b/pkg/rpc/clock_offset_test.go index 949df5727066..2bc452029f7b 100644 --- a/pkg/rpc/clock_offset_test.go +++ b/pkg/rpc/clock_offset_test.go @@ -50,7 +50,6 @@ func TestUpdateOffset(t *testing.T) { } else if o != offset1 { t.Errorf("expected offset %v, instead %v", offset1, o) } - // nolint:deferunlock monitor.mu.Unlock() // Case 2: The old offset for addr is stale. @@ -66,7 +65,6 @@ func TestUpdateOffset(t *testing.T) { } else if o != offset2 { t.Errorf("expected offset %v, instead %v", offset2, o) } - // nolint:deferunlock monitor.mu.Unlock() // Case 3: The new offset's error is smaller. @@ -82,7 +80,6 @@ func TestUpdateOffset(t *testing.T) { } else if o != offset3 { t.Errorf("expected offset %v, instead %v", offset3, o) } - // nolint:deferunlock monitor.mu.Unlock() // Larger error and offset3 is not stale, so no update. @@ -93,7 +90,6 @@ func TestUpdateOffset(t *testing.T) { } else if o != offset3 { t.Errorf("expected offset %v, instead %v", offset3, o) } - // nolint:deferunlock monitor.mu.Unlock() } diff --git a/pkg/rpc/context.go b/pkg/rpc/context.go index 3b1030f9f6f2..e18cf789bacb 100644 --- a/pkg/rpc/context.go +++ b/pkg/rpc/context.go @@ -537,7 +537,6 @@ func NewContext(ctx context.Context, opts ContextOptions) *Context { rpcCtx.dialbackMu.Lock() rpcCtx.dialbackMu.m = map[roachpb.NodeID]*Connection{} - // nolint:deferunlock rpcCtx.dialbackMu.Unlock() if !opts.TenantID.IsSet() { diff --git a/pkg/rpc/context_test.go b/pkg/rpc/context_test.go index 4ac948b9661e..ee69efc21c6f 100644 --- a/pkg/rpc/context_test.go +++ b/pkg/rpc/context_test.go @@ -1106,7 +1106,6 @@ func (ac *AdvancingClock) Now() time.Time { ac.Lock() now := ac.time ac.time = now.Add(ac.getAdvancementInterval()) - // nolint:deferunlock ac.Unlock() return now } diff --git a/pkg/rpc/nodedialer/nodedialer_test.go b/pkg/rpc/nodedialer/nodedialer_test.go index 6e7ca412241e..1ef4ad3a48a1 100644 --- a/pkg/rpc/nodedialer/nodedialer_test.go +++ b/pkg/rpc/nodedialer/nodedialer_test.go @@ -353,7 +353,6 @@ func (il *interceptingListener) Accept() (c net.Conn, err error) { if err == nil { il.mu.Lock() il.mu.conns = append(il.mu.conns, c) - // nolint:deferunlock il.mu.Unlock() } }() diff --git a/pkg/rpc/peer.go b/pkg/rpc/peer.go index d320eb4cbf35..e137d8ba61c4 100644 --- a/pkg/rpc/peer.go +++ b/pkg/rpc/peer.go @@ -232,7 +232,6 @@ func (p *peer) launch(ctx context.Context, report func(error), done func()) { // to not hold the lock. p.mu.Lock() _ = 0 // bypass empty crit section lint - // nolint:deferunlock p.mu.Unlock() taskName := fmt.Sprintf("conn to n%d@%s/%s", p.k.NodeID, p.k.TargetAddr, p.k.Class) @@ -303,7 +302,6 @@ func (p *peer) run(ctx context.Context, report func(error), done func()) { p.mu.Lock() p.mu.c = newConnectionToNodeID(p.k, p.mu.c.breakerSignalFn) - // nolint:deferunlock p.mu.Unlock() if p.snap().deleteAfter != 0 { @@ -814,7 +812,6 @@ func touchOldPeers(peers *peerMap, now time.Time) { sigs = append(sigs, p.b.Signal()) } } - // nolint:deferunlock peers.mu.RUnlock() // Now, outside of the lock, query all of the collected Signals which will tip diff --git a/pkg/security/auth.go b/pkg/security/auth.go index c6a9decb49cc..5da50ec86132 100644 --- a/pkg/security/auth.go +++ b/pkg/security/auth.go @@ -77,7 +77,6 @@ func SetCertPrincipalMap(mappings []string) error { } certPrincipalMap.Lock() certPrincipalMap.m = m - // nolint:deferunlock certPrincipalMap.Unlock() return nil } @@ -85,7 +84,6 @@ func SetCertPrincipalMap(mappings []string) error { func transformPrincipal(commonName string) string { certPrincipalMap.RLock() mappedName, ok := certPrincipalMap.m[commonName] - // nolint:deferunlock certPrincipalMap.RUnlock() if !ok { return commonName diff --git a/pkg/server/drain.go b/pkg/server/drain.go index 8f1aac3b3b29..8e1623067333 100644 --- a/pkg/server/drain.go +++ b/pkg/server/drain.go @@ -292,7 +292,6 @@ func (s *drainServer) runDrain( if howMany > 0 { mu.Lock() reports[what] += howMany - // nolint:deferunlock mu.Unlock() } } diff --git a/pkg/server/init.go b/pkg/server/init.go index 7cf3c95b9276..f371b85563c0 100644 --- a/pkg/server/init.go +++ b/pkg/server/init.go @@ -414,7 +414,6 @@ func (s *initServer) startJoinLoop(ctx context.Context, stopper *stop.Stopper) ( // We mark ourselves as bootstrapped to prevent future bootstrap attempts. s.mu.Lock() s.mu.bootstrapped = true - // nolint:deferunlock s.mu.Unlock() return state, nil @@ -463,7 +462,6 @@ func (s *initServer) startJoinLoop(ctx context.Context, stopper *stop.Stopper) ( // We mark ourselves as bootstrapped to prevent future bootstrap attempts. s.mu.Lock() s.mu.bootstrapped = true - // nolint:deferunlock s.mu.Unlock() return state, nil diff --git a/pkg/server/node.go b/pkg/server/node.go index 2989c5a66e99..5b59a14d8750 100644 --- a/pkg/server/node.go +++ b/pkg/server/node.go @@ -1753,7 +1753,6 @@ func newMuxRangeFeedCompletionWatcher( var toSend []*kvpb.MuxRangeFeedEvent fin.Lock() toSend, fin.completed = fin.completed, nil - // nolint:deferunlock fin.Unlock() for _, e := range toSend { if err := send(e); err != nil { @@ -1785,7 +1784,6 @@ func newMuxRangeFeedCompletionWatcher( addCompleted := func(event *kvpb.MuxRangeFeedEvent) { fin.Lock() fin.completed = append(fin.completed, event) - // nolint:deferunlock fin.Unlock() select { case fin.signalC <- struct{}{}: diff --git a/pkg/server/node_tombstone_storage.go b/pkg/server/node_tombstone_storage.go index 0bca7a744624..9d7a576ef247 100644 --- a/pkg/server/node_tombstone_storage.go +++ b/pkg/server/node_tombstone_storage.go @@ -50,7 +50,6 @@ func (s *nodeTombstoneStorage) IsDecommissioned( ) (time.Time, error) { s.mu.RLock() ts, ok := s.mu.cache[nodeID] - // nolint:deferunlock s.mu.RUnlock() if ok { // Cache hit. diff --git a/pkg/server/profiler/activequeryprofiler_test.go b/pkg/server/profiler/activequeryprofiler_test.go index d5b64e6a9a3e..635d4e0d4778 100644 --- a/pkg/server/profiler/activequeryprofiler_test.go +++ b/pkg/server/profiler/activequeryprofiler_test.go @@ -188,7 +188,6 @@ func TestShouldDump(t *testing.T) { profiler.mu.Lock() profiler.mu.prevMemUsage = test.prevUsage - // nolint:deferunlock profiler.mu.Unlock() memUsageFn = test.memUsageFn diff --git a/pkg/server/settings_cache.go b/pkg/server/settings_cache.go index a79ebd8839c5..d3110d6ed6a9 100644 --- a/pkg/server/settings_cache.go +++ b/pkg/server/settings_cache.go @@ -70,13 +70,11 @@ func (s *settingsCacheWriter) queueSnapshot(kvs []roachpb.KeyValue) (shouldRun b s.mu.Lock() // held into the async task if s.mu.currentlyWriting { s.mu.queuedToWrite = kvs - // nolint:deferunlock s.mu.Unlock() return false } s.mu.currentlyWriting = true s.mu.queuedToWrite = kvs - // nolint:deferunlock s.mu.Unlock() return true } diff --git a/pkg/server/telemetry/features.go b/pkg/server/telemetry/features.go index 7b2d25abf2b3..d1963a339673 100644 --- a/pkg/server/telemetry/features.go +++ b/pkg/server/telemetry/features.go @@ -79,7 +79,6 @@ func Read(c Counter) int32 { func GetCounterOnce(feature string) Counter { counters.RLock() _, ok := counters.m[feature] - // nolint:deferunlock counters.RUnlock() if ok { panic("counter already exists: " + feature) @@ -91,7 +90,6 @@ func GetCounterOnce(feature string) Counter { func GetCounter(feature string) Counter { counters.RLock() i, ok := counters.m[feature] - // nolint:deferunlock counters.RUnlock() if ok { return i diff --git a/pkg/server/tenantsettingswatcher/overrides_store.go b/pkg/server/tenantsettingswatcher/overrides_store.go index 298785871a1c..dd8afb7d1b7a 100644 --- a/pkg/server/tenantsettingswatcher/overrides_store.go +++ b/pkg/server/tenantsettingswatcher/overrides_store.go @@ -105,7 +105,6 @@ func (s *overridesStore) SetAll(allOverrides map[roachpb.TenantID][]kvpb.TenantS func (s *overridesStore) GetTenantOverrides(tenantID roachpb.TenantID) *tenantOverrides { s.mu.RLock() res, ok := s.mu.tenants[tenantID] - // nolint:deferunlock s.mu.RUnlock() if ok { return res diff --git a/pkg/settings/values.go b/pkg/settings/values.go index 4c33702dea1d..8e9bd0c7ac6a 100644 --- a/pkg/settings/values.go +++ b/pkg/settings/values.go @@ -144,7 +144,6 @@ func (sv *Values) Opaque() interface{} { func (sv *Values) settingChanged(ctx context.Context, slot slotIdx) { sv.changeMu.Lock() funcs := sv.changeMu.onChange[slot] - // nolint:deferunlock sv.changeMu.Unlock() for _, fn := range funcs { fn(ctx) @@ -203,6 +202,5 @@ func (sv *Values) getGeneric(slot slotIdx) interface{} { func (sv *Values) setOnChange(slot slotIdx, fn func(ctx context.Context)) { sv.changeMu.Lock() sv.changeMu.onChange[slot] = append(sv.changeMu.onChange[slot], fn) - // nolint:deferunlock sv.changeMu.Unlock() } diff --git a/pkg/spanconfig/spanconfigjob/job.go b/pkg/spanconfig/spanconfigjob/job.go index ee943abbe1c9..089be902ea6e 100644 --- a/pkg/spanconfig/spanconfigjob/job.go +++ b/pkg/spanconfig/spanconfigjob/job.go @@ -172,7 +172,6 @@ func (r *resumer) Resume(ctx context.Context, execCtxI interface{}) (jobErr erro persistCheckpointsMu.Lock() shouldPersistCheckpoint := persistCheckpointsMu.ShouldProcess(timeutil.Now()) - // nolint:deferunlock persistCheckpointsMu.Unlock() if !shouldPersistCheckpoint { return nil diff --git a/pkg/spanconfig/spanconfigkvsubscriber/datadriven_test.go b/pkg/spanconfig/spanconfigkvsubscriber/datadriven_test.go index 3ed228e2d479..e180fc2e87a0 100644 --- a/pkg/spanconfig/spanconfigkvsubscriber/datadriven_test.go +++ b/pkg/spanconfig/spanconfigkvsubscriber/datadriven_test.go @@ -206,7 +206,6 @@ func TestDataDriven(t *testing.T) { case "start": mu.Lock() require.False(t, mu.subscriberRunning, "subscriber already running") - // nolint:deferunlock mu.Unlock() go func() { @@ -247,7 +246,6 @@ func TestDataDriven(t *testing.T) { mu.Lock() receivedUpdates := mu.receivedUpdates mu.receivedUpdates = mu.receivedUpdates[:0] // clear out buffer - // nolint:deferunlock mu.Unlock() var output strings.Builder diff --git a/pkg/spanconfig/spanconfigkvsubscriber/kvsubscriber.go b/pkg/spanconfig/spanconfigkvsubscriber/kvsubscriber.go index fa8fdbbf4286..43f69e7da7ca 100644 --- a/pkg/spanconfig/spanconfigkvsubscriber/kvsubscriber.go +++ b/pkg/spanconfig/spanconfigkvsubscriber/kvsubscriber.go @@ -419,7 +419,6 @@ func (s *KVSubscriber) handleCompleteUpdate( s.mu.internal = freshStore s.setLastUpdatedLocked(ts) handlers := s.mu.handlers - // nolint:deferunlock s.mu.Unlock() for i := range handlers { handler := &handlers[i] // mutated by invoke @@ -445,7 +444,6 @@ func (s *KVSubscriber) handlePartialUpdate( } s.setLastUpdatedLocked(ts) handlers := s.mu.handlers - // nolint:deferunlock s.mu.Unlock() for i := range handlers { diff --git a/pkg/spanconfig/spanconfigmanager/manager_test.go b/pkg/spanconfig/spanconfigmanager/manager_test.go index d93a9b1ae787..62069dd297e0 100644 --- a/pkg/spanconfig/spanconfigmanager/manager_test.go +++ b/pkg/spanconfig/spanconfigmanager/manager_test.go @@ -351,14 +351,12 @@ func TestReconciliationJobErrorAndRecovery(t *testing.T) { mu.Lock() mu.err = errors.New("injected") - // nolint:deferunlock mu.Unlock() waitForJobStatus(t, tdb, jobID, jobs.StatusFailed) mu.Lock() mu.err = nil - // nolint:deferunlock mu.Unlock() started, err = manager.TestingCreateAndStartJobIfNoneExists(ctx) @@ -369,7 +367,6 @@ func TestReconciliationJobErrorAndRecovery(t *testing.T) { mu.Lock() require.True(t, mu.lastStartTS.IsEmpty(), "expected reconciler to start with empty checkpoint") - // nolint:deferunlock mu.Unlock() } @@ -437,7 +434,6 @@ func TestReconciliationUsesRightCheckpoint(t *testing.T) { mu.Lock() require.True(t, mu.lastStartTS.IsEmpty()) - // nolint:deferunlock mu.Unlock() // Force an internal retry. diff --git a/pkg/spanconfig/spanconfigreconciler/reconciler.go b/pkg/spanconfig/spanconfigreconciler/reconciler.go index d4598110e210..a1e5050814bd 100644 --- a/pkg/spanconfig/spanconfigreconciler/reconciler.go +++ b/pkg/spanconfig/spanconfigreconciler/reconciler.go @@ -168,7 +168,6 @@ func (r *Reconciler) Reconcile( r.mu.Lock() r.mu.lastCheckpoint = reconciledUpUntil - // nolint:deferunlock r.mu.Unlock() if err := onCheckpoint(); err != nil { @@ -189,7 +188,6 @@ func (r *Reconciler) Reconcile( return incremental.reconcile(ctx, incrementalStartTS, func(reconciledUpUntil hlc.Timestamp) error { r.mu.Lock() r.mu.lastCheckpoint = reconciledUpUntil - // nolint:deferunlock r.mu.Unlock() return onCheckpoint() diff --git a/pkg/sql/catalog/lease/lease_internal_test.go b/pkg/sql/catalog/lease/lease_internal_test.go index a22b0f3a3538..9a2daa7bc748 100644 --- a/pkg/sql/catalog/lease/lease_internal_test.go +++ b/pkg/sql/catalog/lease/lease_internal_test.go @@ -215,7 +215,6 @@ CREATE TABLE t.test (k CHAR PRIMARY KEY, v CHAR); correctLease := ts.mu.active.data[0].GetID() == tables[5].GetID() && ts.mu.active.data[0].GetVersion() == tables[5].GetVersion() correctExpiration := ts.mu.active.data[0].getExpiration() == expiration - // nolint:deferunlock ts.mu.Unlock() if !correctLease { t.Fatalf("wrong lease survived purge") @@ -232,7 +231,6 @@ CREATE TABLE t.test (k CHAR PRIMARY KEY, v CHAR); } tableVersion.mu.expiration = tables[5].GetModificationTime() ts.mu.active.insert(tableVersion) - // nolint:deferunlock ts.mu.Unlock() if numLeases := getNumVersions(ts); numLeases != 2 { t.Fatalf("found %d versions instead of 2", numLeases) @@ -352,7 +350,6 @@ CREATE TABLE t.test (k CHAR PRIMARY KEY, v CHAR); ts.mu.Lock() correctLease := ts.mu.active.data[0].GetID() == latestDesc.GetID() && ts.mu.active.data[0].GetVersion() == latestDesc.GetVersion() - // nolint:deferunlock ts.mu.Unlock() if !correctLease { t.Fatalf("wrong lease survived purge") @@ -1159,7 +1156,6 @@ func TestReadOlderVersionForTimestamp(t *testing.T) { } else { addedDescVState.mu.expiration = hlc.MaxTimestamp } - // nolint:deferunlock addedDescVState.mu.Unlock() descStates[tableID].mu.active.insert(addedDescVState) } diff --git a/pkg/sql/catalog/lease/lease_test.go b/pkg/sql/catalog/lease/lease_test.go index 9fea13a5df5a..9f47cbf1db09 100644 --- a/pkg/sql/catalog/lease/lease_test.go +++ b/pkg/sql/catalog/lease/lease_test.go @@ -622,7 +622,6 @@ CREATE TABLE test.t(a INT PRIMARY KEY); // dropped; it will be left in a "deleted" state. mu.Lock() clearSchemaChangers = true - // nolint:deferunlock mu.Unlock() // DROP the table @@ -729,7 +728,6 @@ CREATE TABLE test.t(a INT PRIMARY KEY); mu.Lock() clearSchemaChangers = true waitTableID = tableDesc.GetID() - // nolint:deferunlock mu.Unlock() // DROP the table @@ -2829,7 +2827,6 @@ func TestOfflineLeaseRefresh(t *testing.T) { TestingRequestFilter: func(ctx context.Context, req *kvpb.BatchRequest) *kvpb.Error { mu.RLock() checkRequest := req.Txn != nil && req.Txn.ID.Equal(txnID) - // nolint:deferunlock mu.RUnlock() if _, ok := req.GetArg(kvpb.EndTxn); checkRequest && ok { notify := make(chan struct{}) @@ -2884,7 +2881,6 @@ CREATE TABLE d1.t2 (name int); mu.Lock() waitForRqstFilter = make(chan chan struct{}) txnID = txn.KV().ID() - // nolint:deferunlock mu.Unlock() // Online the descriptor by making it public @@ -2920,7 +2916,6 @@ CREATE TABLE d1.t2 (name int); close(notify) mu.RLock() rqstFilterChannel := waitForRqstFilter - // nolint:deferunlock mu.RUnlock() for notify2 := range rqstFilterChannel { // Push the query trying to online the table out by @@ -2961,7 +2956,6 @@ func TestLeaseTxnDeadlineExtension(t *testing.T) { // Wait for a commit with the txnID, and only allows // it to resume when the channel gets unblocked. if req.Txn != nil && req.Txn.ID.String() == txnID { - // nolint:deferunlock filterMu.Unlock() // There will only be a single EndTxn request in // flight due to the transaction ID filter and @@ -3014,7 +3008,6 @@ SELECT * FROM t1; filterMu.Lock() err = txnIDResult.Scan(&txnID) blockedOnce = false - // nolint:deferunlock filterMu.Unlock() if err != nil { waitChan <- err @@ -3077,7 +3070,6 @@ SELECT * FROM t1; filterMu.Lock() err = txnIDResult.Scan(&txnID) blockedOnce = false - // nolint:deferunlock filterMu.Unlock() if err != nil { waitChan <- err @@ -3155,7 +3147,6 @@ func TestLeaseBulkInsertWithImplicitTxn(t *testing.T) { beforeExecute.Lock() if stmt == beforeExecuteStmt { tableID := descpb.ID(atomic.LoadUint64(&leaseTableID)) - // nolint:deferunlock beforeExecute.Unlock() waitChan := make(chan struct{}) select { @@ -3181,14 +3172,12 @@ func TestLeaseBulkInsertWithImplicitTxn(t *testing.T) { AfterExecute: func(ctx context.Context, stmt string, err error) { beforeExecute.Lock() if stmt == beforeExecuteResumeStmt { - // nolint:deferunlock beforeExecute.Unlock() resumeChan, ok := <-beforeExecuteWait if ok { close(resumeChan) } } else { - // nolint:deferunlock beforeExecute.Unlock() } }, @@ -3233,7 +3222,6 @@ INSERT INTO t1 select a from generate_series(1, 100) g(a); const bulkUpdateQuery = "UPDATE t1 SET val = 2" beforeExecute.Lock() beforeExecuteStmt = bulkUpdateQuery - // nolint:deferunlock beforeExecute.Unlock() // Execute a bulk UPDATE, which will get its // timestamp pushed by a read operation. @@ -3249,7 +3237,6 @@ INSERT INTO t1 select a from generate_series(1, 100) g(a); ) beforeExecute.Lock() beforeExecuteResumeStmt = selectStmt - // nolint:deferunlock beforeExecute.Unlock() // While the update hasn't completed executing, repeatedly // execute selects to push out the update operation. We will @@ -3266,7 +3253,6 @@ INSERT INTO t1 select a from generate_series(1, 100) g(a); // like normal after being pushed a limited number of times. beforeExecute.Lock() beforeExecuteStmt, beforeExecuteResumeStmt = "", "" - // nolint:deferunlock beforeExecute.Unlock() resumeChan, channelReadOk := <-beforeExecuteWait if channelReadOk { @@ -3438,7 +3424,6 @@ func TestDescriptorRemovedFromCacheWhenLeaseRenewalForThisDescriptorFails(t *tes mu.Lock() typeDescID = typeDesc.GetID() typeDescName = typeDesc.GetName() - // nolint:deferunlock mu.Unlock() // Wait until the testing knob drops `typ` diff --git a/pkg/sql/catalog/lease/name_cache.go b/pkg/sql/catalog/lease/name_cache.go index 26e3b2350325..ee73a7266fb2 100644 --- a/pkg/sql/catalog/lease/name_cache.go +++ b/pkg/sql/catalog/lease/name_cache.go @@ -52,7 +52,6 @@ func (c *nameCache) get( desc, ok := c.descriptors.GetByName( parentID, parentSchemaID, name, ).(*descriptorVersionState) - // nolint:deferunlock c.mu.RUnlock() if !ok { return nil @@ -60,7 +59,6 @@ func (c *nameCache) get( expensiveLogEnabled := log.ExpensiveLogEnabled(ctx, 2) desc.mu.Lock() if desc.mu.lease == nil { - // nolint:deferunlock desc.mu.Unlock() // This get() raced with a release operation. Remove this cache // entry if needed. diff --git a/pkg/sql/colfetcher/colbatch_direct_scan.go b/pkg/sql/colfetcher/colbatch_direct_scan.go index a97f9978818d..058f72404671 100644 --- a/pkg/sql/colfetcher/colbatch_direct_scan.go +++ b/pkg/sql/colfetcher/colbatch_direct_scan.go @@ -99,7 +99,6 @@ func (s *ColBatchDirectScan) Next() (ret coldata.Batch) { } s.mu.Lock() s.mu.rowsRead += int64(res.ColBatch.Length()) - // nolint:deferunlock s.mu.Unlock() // Note that this batch has already been accounted for by the // KVBatchFetcher, so we don't need to do that. @@ -122,7 +121,6 @@ func (s *ColBatchDirectScan) Next() (ret coldata.Batch) { batch := s.deserializer.Deserialize(res.BatchResponse) s.mu.Lock() s.mu.rowsRead += int64(batch.Length()) - // nolint:deferunlock s.mu.Unlock() return batch } diff --git a/pkg/sql/colfetcher/colbatch_scan.go b/pkg/sql/colfetcher/colbatch_scan.go index a768736b982b..199eeb7c4c0f 100644 --- a/pkg/sql/colfetcher/colbatch_scan.go +++ b/pkg/sql/colfetcher/colbatch_scan.go @@ -264,7 +264,6 @@ func (s *ColBatchScan) Next() coldata.Batch { } s.mu.Lock() s.mu.rowsRead += int64(bat.Length()) - // nolint:deferunlock s.mu.Unlock() return bat } diff --git a/pkg/sql/colfetcher/index_join.go b/pkg/sql/colfetcher/index_join.go index 47aaf017b962..ea66b7f16f2d 100644 --- a/pkg/sql/colfetcher/index_join.go +++ b/pkg/sql/colfetcher/index_join.go @@ -247,7 +247,6 @@ func (s *ColIndexJoin) Next() coldata.Batch { } s.mu.Lock() s.mu.rowsRead += int64(n) - // nolint:deferunlock s.mu.Unlock() return batch case indexJoinDone: diff --git a/pkg/sql/colflow/routers.go b/pkg/sql/colflow/routers.go index 09ece02cb62f..be4bc81ceebb 100644 --- a/pkg/sql/colflow/routers.go +++ b/pkg/sql/colflow/routers.go @@ -280,7 +280,6 @@ func (o *routerOutputOp) DrainMeta() []execinfrapb.ProducerMetadata { // The call to DrainMeta() indicates that the caller will no longer need any // more data from this output, so we can close it. o.closeLocked(o.Ctx) - // nolint:deferunlock o.mu.Unlock() return o.drainCoordinator.drainMeta() } diff --git a/pkg/sql/colflow/routers_test.go b/pkg/sql/colflow/routers_test.go index 67e89c59a5d6..48cb36a5d421 100644 --- a/pkg/sql/colflow/routers_test.go +++ b/pkg/sql/colflow/routers_test.go @@ -1156,7 +1156,6 @@ func TestHashRouterRandom(t *testing.T) { if meta := outputsAsOps[i].DrainMeta(); meta != nil { metadataMu.metadata = append(metadataMu.metadata, meta) } - // nolint:deferunlock metadataMu.Unlock() break } diff --git a/pkg/sql/colflow/stats.go b/pkg/sql/colflow/stats.go index 7f8522c229dc..0fa0d66130bb 100644 --- a/pkg/sql/colflow/stats.go +++ b/pkg/sql/colflow/stats.go @@ -111,7 +111,6 @@ func (bic *batchInfoCollector) Init(ctx context.Context) { // If we got here, then Init above succeeded, so the wrapped operator has // been properly initialized. bic.mu.initialized = true - // nolint:deferunlock bic.mu.Unlock() } @@ -133,7 +132,6 @@ func (bic *batchInfoCollector) Next() coldata.Batch { bic.mu.Lock() bic.mu.numBatches++ bic.mu.numTuples += uint64(bic.batch.Length()) - // nolint:deferunlock bic.mu.Unlock() } return bic.batch diff --git a/pkg/sql/colflow/vectorized_flow.go b/pkg/sql/colflow/vectorized_flow.go index 28dd147d1dda..6bcd7e6472b3 100644 --- a/pkg/sql/colflow/vectorized_flow.go +++ b/pkg/sql/colflow/vectorized_flow.go @@ -386,7 +386,6 @@ func (f *vectorizedFlow) Cleanup(ctx context.Context) { f.tempStorage.Lock() created := f.tempStorage.path != "" - // nolint:deferunlock f.tempStorage.Unlock() if created { if err := f.Cfg.TempFS.RemoveAll(f.GetPath(ctx)); err != nil { diff --git a/pkg/sql/conn_executor_exec.go b/pkg/sql/conn_executor_exec.go index 446e648a317a..886b249b3143 100644 --- a/pkg/sql/conn_executor_exec.go +++ b/pkg/sql/conn_executor_exec.go @@ -3064,7 +3064,6 @@ func (ex *connExecutor) recordTransactionStart(txnID uuid.UUID) { ex.state.mu.RLock() txnStart := ex.state.mu.txnStart - // nolint:deferunlock ex.state.mu.RUnlock() ex.phaseTimes.SetSessionPhaseTime(sessionphase.SessionTransactionStarted, txnStart) diff --git a/pkg/sql/contention/contentionutils/concurrent_buffer_guard_test.go b/pkg/sql/contention/contentionutils/concurrent_buffer_guard_test.go index e7ccd29004d2..b8a5b62106d3 100644 --- a/pkg/sql/contention/contentionutils/concurrent_buffer_guard_test.go +++ b/pkg/sql/contention/contentionutils/concurrent_buffer_guard_test.go @@ -68,7 +68,6 @@ func newTestBuffer(sizeLimit int64) *testAsyncBuffer { p := t.writerBuffer[idx] t.validation.readMap[p.k] = p.v } - // nolint:deferunlock t.validation.Unlock() // Resets t.writerBuffer. diff --git a/pkg/sql/contention/event_store.go b/pkg/sql/contention/event_store.go index a4dd788e20cb..2079dc137314 100644 --- a/pkg/sql/contention/event_store.go +++ b/pkg/sql/contention/event_store.go @@ -269,7 +269,6 @@ func (s *eventStore) ForEachEvent( s.mu.store.Do(func(entry *cache.Entry) { keys = append(keys, entry.Key.(uint64)) }) - // nolint:deferunlock s.mu.RUnlock() for i := range keys { diff --git a/pkg/sql/distsql_running_test.go b/pkg/sql/distsql_running_test.go index 65b54e67f139..938a3c344ff0 100644 --- a/pkg/sql/distsql_running_test.go +++ b/pkg/sql/distsql_running_test.go @@ -385,7 +385,6 @@ func TestDistSQLRunningParallelFKChecksAfterAbort(t *testing.T) { // clear the abortTxn function before returning. mu.Lock() mu.abortTxn = nil - // nolint:deferunlock mu.Unlock() }() } diff --git a/pkg/sql/flowinfra/flow_registry.go b/pkg/sql/flowinfra/flow_registry.go index 4353af150700..224fe3a8baca 100644 --- a/pkg/sql/flowinfra/flow_registry.go +++ b/pkg/sql/flowinfra/flow_registry.go @@ -349,7 +349,6 @@ func (fr *FlowRegistry) cancelPendingStreams( // FlowRegistry mutex. inboundStreams = entry.inboundStreams } - // nolint:deferunlock fr.Unlock() if inboundStreams == nil { return 0 @@ -459,7 +458,6 @@ func (fr *FlowRegistry) Drain( fr.Lock() fr.draining = true if len(fr.flows) > 0 { - // nolint:deferunlock fr.Unlock() time.Sleep(expectedConnectionTime) fr.Lock() @@ -478,13 +476,11 @@ func (fr *FlowRegistry) Drain( fr.Lock() if len(fr.flows) == 0 { - // nolint:deferunlock fr.Unlock() sleep(minFlowDrainWait) fr.Lock() // No flows were registered, return. if len(fr.flows) == 0 { - // nolint:deferunlock fr.Unlock() return } @@ -500,7 +496,6 @@ func (fr *FlowRegistry) Drain( fr.Lock() stopWaiting = true fr.flowDone.Signal() - // nolint:deferunlock fr.Unlock() case <-allFlowsDone: } @@ -521,7 +516,6 @@ func (fr *FlowRegistry) Drain( for !(stopWaiting || len(fr.flows) == 0) { fr.flowDone.Wait() } - // nolint:deferunlock fr.Unlock() } @@ -532,7 +526,6 @@ func (fr *FlowRegistry) Drain( func (fr *FlowRegistry) undrain() { fr.Lock() fr.draining = false - // nolint:deferunlock fr.Unlock() } @@ -567,7 +560,6 @@ func (fr *FlowRegistry) ConnectInboundStream( // below one; in all other cases we want to make sure to delete the entry // from the registry if we're holding the only reference. entry.refCount++ - // nolint:deferunlock fr.Unlock() defer func() { fr.Lock() diff --git a/pkg/sql/flowinfra/flow_registry_test.go b/pkg/sql/flowinfra/flow_registry_test.go index 2f930f0a65c4..b2a54ce3b31b 100644 --- a/pkg/sql/flowinfra/flow_registry_test.go +++ b/pkg/sql/flowinfra/flow_registry_test.go @@ -40,7 +40,6 @@ func lookupFlow(fr *FlowRegistry, fid execinfrapb.FlowID, timeout time.Duration) fr.Lock() entry := fr.getEntryLocked(fid) flow := entry.flow - // nolint:deferunlock fr.Unlock() if flow == nil { flow = fr.waitForFlow(context.Background(), fid, timeout) @@ -56,7 +55,6 @@ func lookupStreamInfo( fr.Lock() entry := fr.getEntryLocked(fid) flowFound := entry.flow != nil - // nolint:deferunlock fr.Unlock() if !flowFound { return nil, errors.Errorf("missing flow entry: %s", fid) diff --git a/pkg/sql/idxusage/local_idx_usage_stats.go b/pkg/sql/idxusage/local_idx_usage_stats.go index 7f78c05ca10d..028cd71cebd3 100644 --- a/pkg/sql/idxusage/local_idx_usage_stats.go +++ b/pkg/sql/idxusage/local_idx_usage_stats.go @@ -179,7 +179,6 @@ func (s *LocalIndexUsageStats) ForEach(options IteratorOptions, visitor StatsVis for tableID := range s.mu.usageStats { tableIDLists = append(tableIDLists, tableID) } - // nolint:deferunlock s.mu.RUnlock() if options.SortedTableID { @@ -324,12 +323,10 @@ func (t *tableIndexStats) getStatsForIndexID( t.RLock() if stats, ok := t.stats[id]; ok || !createIfNotExists { - // nolint:deferunlock t.RUnlock() return stats } - // nolint:deferunlock t.RUnlock() t.Lock() defer t.Unlock() @@ -378,7 +375,6 @@ func (t *tableIndexStats) iterateIndexStats( indexIDs = append(indexIDs, indexID) iterLimit-- } - // nolint:deferunlock t.RUnlock() if orderedIndexID { @@ -399,7 +395,6 @@ func (t *tableIndexStats) iterateIndexStats( indexStats.RLock() // Copy out the stats while holding read lock. statsCopy := indexStats.IndexUsageStatistics - // nolint:deferunlock indexStats.RUnlock() if err := visitor(&roachpb.IndexUsageKey{ diff --git a/pkg/sql/importer/exportcsv_test.go b/pkg/sql/importer/exportcsv_test.go index 9ec27a0c4569..3ab909538b1d 100644 --- a/pkg/sql/importer/exportcsv_test.go +++ b/pkg/sql/importer/exportcsv_test.go @@ -582,7 +582,6 @@ func TestProcessorEncountersUncertaintyError(t *testing.T) { // already been blocked. blockedRead.shouldUnblock = true blockedRead.unblockCond.Signal() - // nolint:deferunlock blockedRead.Unlock() } @@ -591,7 +590,6 @@ func TestProcessorEncountersUncertaintyError(t *testing.T) { for !blockedRead.shouldUnblock { blockedRead.unblockCond.Wait() } - // nolint:deferunlock blockedRead.Unlock() } diff --git a/pkg/sql/importer/import_processor.go b/pkg/sql/importer/import_processor.go index f3a5baedf8fc..d0c162cc927a 100644 --- a/pkg/sql/importer/import_processor.go +++ b/pkg/sql/importer/import_processor.go @@ -446,7 +446,6 @@ func ingestKvs( atomic.StoreInt64(&pkFlushedRow[i], emitted) bulkSummaryMu.Lock() bulkSummaryMu.summary.Add(summary) - // nolint:deferunlock bulkSummaryMu.Unlock() } if indexAdder.IsEmpty() { @@ -460,7 +459,6 @@ func ingestKvs( atomic.StoreInt64(&idxFlushedRow[i], emitted) bulkSummaryMu.Lock() bulkSummaryMu.summary.Add(summary) - // nolint:deferunlock bulkSummaryMu.Unlock() } }) @@ -492,7 +490,6 @@ func ingestKvs( bulkSummaryMu.Lock() prog.BulkSummary = bulkSummaryMu.summary bulkSummaryMu.summary.Reset() - // nolint:deferunlock bulkSummaryMu.Unlock() } select { diff --git a/pkg/sql/importer/import_processor_planning.go b/pkg/sql/importer/import_processor_planning.go index 96c6df00b468..86ea46d3a5af 100644 --- a/pkg/sql/importer/import_processor_planning.go +++ b/pkg/sql/importer/import_processor_planning.go @@ -132,7 +132,6 @@ func distImport( }{} accumulatedBulkSummary.Lock() accumulatedBulkSummary.BulkOpSummary = getLastImportSummary(job) - // nolint:deferunlock accumulatedBulkSummary.Unlock() importDetails := job.Progress().Details.(*jobspb.Progress_Import).Import @@ -179,7 +178,6 @@ func distImport( accumulatedBulkSummary.Lock() prog.Summary.Add(accumulatedBulkSummary.BulkOpSummary) accumulatedBulkSummary.Reset() - // nolint:deferunlock accumulatedBulkSummary.Unlock() return overall / float32(len(from)) }, @@ -197,7 +195,6 @@ func distImport( accumulatedBulkSummary.Lock() accumulatedBulkSummary.Add(meta.BulkProcessorProgress.BulkSummary) - // nolint:deferunlock accumulatedBulkSummary.Unlock() if testingKnobs.alwaysFlushJobProgress { diff --git a/pkg/sql/importer/import_processor_test.go b/pkg/sql/importer/import_processor_test.go index a38996c6c52c..f550b3d7d518 100644 --- a/pkg/sql/importer/import_processor_test.go +++ b/pkg/sql/importer/import_processor_test.go @@ -412,7 +412,6 @@ func TestImportHonorsResumePosition(t *testing.T) { pkBulkAdder.onKeyAdd = func(k roachpb.Key) { keys.Lock() keys.keys = append(keys.keys, k) - // nolint:deferunlock keys.Unlock() } } else { @@ -433,7 +432,6 @@ func TestImportHonorsResumePosition(t *testing.T) { if idx < maxKeyIdx { t.Errorf("failed to skip key[%d]=%s", idx, k) } - // nolint:deferunlock keys.Unlock() } } @@ -470,7 +468,6 @@ func TestImportHonorsResumePosition(t *testing.T) { return keys.keys[i].Compare(keys.keys[j]) < 0 }) numKeys = len(keys.keys) - // nolint:deferunlock keys.Unlock() } } diff --git a/pkg/sql/pgwire/hba_conf.go b/pkg/sql/pgwire/hba_conf.go index 7808808642cd..f8a0f6c20afc 100644 --- a/pkg/sql/pgwire/hba_conf.go +++ b/pkg/sql/pgwire/hba_conf.go @@ -320,7 +320,6 @@ func (s *Server) GetAuthenticationConfiguration() (*hba.Conf, *identmap.Conf) { s.auth.RLock() auth := s.auth.conf idMap := s.auth.identityMap - // nolint:deferunlock s.auth.RUnlock() if auth == nil { diff --git a/pkg/sql/pgwire/server.go b/pkg/sql/pgwire/server.go index d566a5cd2003..1ef07dbe5520 100644 --- a/pkg/sql/pgwire/server.go +++ b/pkg/sql/pgwire/server.go @@ -348,7 +348,6 @@ func MakeServer( server.mu.Lock() server.mu.connCancelMap = make(cancelChanMap) - // nolint:deferunlock server.mu.Unlock() connAuthConf.SetOnChange(&st.SV, func(ctx context.Context) { @@ -494,7 +493,6 @@ func (s *Server) WaitForSQLConnsToClose( s.mu.Lock() s.setRejectNewConnectionsLocked(true) - // nolint:deferunlock s.mu.Unlock() if connectionWait == 0 { @@ -1377,7 +1375,6 @@ func (s *Server) registerConn( close(done) s.mu.Lock() delete(s.mu.connCancelMap, done) - // nolint:deferunlock s.mu.Unlock() } } diff --git a/pkg/sql/rename_test.go b/pkg/sql/rename_test.go index 0821a3916e2a..783b27cdfee7 100644 --- a/pkg/sql/rename_test.go +++ b/pkg/sql/rename_test.go @@ -139,7 +139,6 @@ CREATE TABLE test.t (a INT PRIMARY KEY); tableDesc := desctestutils.TestingGetPublicTableDescriptor(kvDB, keys.SystemSQLCodec, "test", "t") mu.Lock() waitTableID = tableDesc.GetID() - // nolint:deferunlock mu.Unlock() txn, err := db.Begin() diff --git a/pkg/sql/rowexec/indexbackfiller.go b/pkg/sql/rowexec/indexbackfiller.go index 182133f20fda..db5df157568b 100644 --- a/pkg/sql/rowexec/indexbackfiller.go +++ b/pkg/sql/rowexec/indexbackfiller.go @@ -218,7 +218,6 @@ func (ib *indexBackfiller) ingestIndexEntries( var prog execinfrapb.RemoteProducerMetadata_BulkProcessorProgress prog.CompletedSpans = append(prog.CompletedSpans, mu.completedSpans...) mu.completedSpans = nil - // nolint:deferunlock mu.Unlock() progCh <- prog @@ -258,7 +257,6 @@ func (ib *indexBackfiller) ingestIndexEntries( // the set of completed spans on the next bulk adder flush. mu.Lock() mu.addedSpans = append(mu.addedSpans, indexBatch.completedSpan) - // nolint:deferunlock mu.Unlock() // After the index KVs have been copied to the underlying BulkAdder, we can diff --git a/pkg/sql/rowexec/processors_test.go b/pkg/sql/rowexec/processors_test.go index 5e88a9acc5ee..e03ec132877b 100644 --- a/pkg/sql/rowexec/processors_test.go +++ b/pkg/sql/rowexec/processors_test.go @@ -485,7 +485,6 @@ func TestDrainingProcessorSwallowsUncertaintyError(t *testing.T) { for !blockedRead.shouldUnblock { blockedRead.unblockCond.Wait() } - // nolint:deferunlock blockedRead.Unlock() return kvpb.NewError( kvpb.NewReadWithinUncertaintyIntervalError( @@ -541,7 +540,6 @@ func TestDrainingProcessorSwallowsUncertaintyError(t *testing.T) { // Reset the blocking condition. blockedRead.Lock() blockedRead.shouldUnblock = false - // nolint:deferunlock blockedRead.Unlock() // Force DistSQL to distribute the query. Otherwise, as of Nov 2018, it's hard // to convince it to distribute a query that uses an index. @@ -588,7 +586,6 @@ func TestDrainingProcessorSwallowsUncertaintyError(t *testing.T) { // already been blocked. blockedRead.shouldUnblock = true blockedRead.unblockCond.Signal() - // nolint:deferunlock blockedRead.Unlock() } } @@ -655,7 +652,6 @@ func TestUncertaintyErrorIsReturned(t *testing.T) { filters[node].Lock() enabled := filters[node].enabled keyPrefix := filters[node].keyPrefix - // nolint:deferunlock filters[node].Unlock() if !enabled { return nil @@ -789,7 +785,6 @@ func TestUncertaintyErrorIsReturned(t *testing.T) { filters[nodeIdx].Lock() filters[nodeIdx].enabled = true filters[nodeIdx].keyPrefix = keys.SystemSQLCodec.TablePrefix(uint32(tableID)) - // nolint:deferunlock filters[nodeIdx].Unlock() } // Reset all filters for the next test case. @@ -798,7 +793,6 @@ func TestUncertaintyErrorIsReturned(t *testing.T) { filters[i].Lock() filters[i].enabled = false filters[i].keyPrefix = nil - // nolint:deferunlock filters[i].Unlock() } }() diff --git a/pkg/sql/rowflow/routers.go b/pkg/sql/rowflow/routers.go index 98a232414555..70dfa6a2cfd6 100644 --- a/pkg/sql/rowflow/routers.go +++ b/pkg/sql/rowflow/routers.go @@ -419,7 +419,6 @@ func (rb *routerBase) ProducerDone() { o := &rb.outputs[i] o.mu.Lock() o.mu.producerDone = true - // nolint:deferunlock o.mu.Unlock() o.mu.cond.Signal() } @@ -474,12 +473,10 @@ func (rb *routerBase) fwdMetadata(meta *execinfrapb.ProducerMetadata) { ro.mu.Lock() if ro.mu.streamStatus != execinfra.ConsumerClosed { ro.addMetadataLocked(meta) - // nolint:deferunlock ro.mu.Unlock() ro.mu.cond.Signal() return } - // nolint:deferunlock ro.mu.Unlock() } } @@ -500,12 +497,10 @@ func (rb *routerBase) fwdErrMetadata(err error) bool { if ro.mu.streamStatus != execinfra.ConsumerClosed { meta := &execinfrapb.ProducerMetadata{Err: err} ro.addMetadataLocked(meta) - // nolint:deferunlock ro.mu.Unlock() ro.mu.cond.Signal() forwarded = true } else { - // nolint:deferunlock ro.mu.Unlock() } } @@ -586,7 +581,6 @@ func (mr *mirrorRouter) Push( ro := &mr.outputs[i] ro.mu.Lock() err := ro.addRowLocked(context.TODO(), row) - // nolint:deferunlock ro.mu.Unlock() if err != nil { if useSema { @@ -643,7 +637,6 @@ func (hr *hashRouter) Push( ro := &hr.outputs[streamIdx] ro.mu.Lock() err = ro.addRowLocked(context.TODO(), row) - // nolint:deferunlock ro.mu.Unlock() ro.mu.cond.Signal() } @@ -729,7 +722,6 @@ func (rr *rangeRouter) Push( ro := &rr.outputs[streamIdx] ro.mu.Lock() err = ro.addRowLocked(context.TODO(), row) - // nolint:deferunlock ro.mu.Unlock() ro.mu.cond.Signal() } diff --git a/pkg/sql/rowflow/routers_test.go b/pkg/sql/rowflow/routers_test.go index 913933fe263a..b6e3b4cb2599 100644 --- a/pkg/sql/rowflow/routers_test.go +++ b/pkg/sql/rowflow/routers_test.go @@ -837,7 +837,6 @@ func TestRouterDiskSpill(t *testing.T) { for _, row := range rows { output.mu.Lock() err := output.addRowLocked(ctx, row) - // nolint:deferunlock output.mu.Unlock() if err != nil { errChan <- err @@ -851,7 +850,6 @@ func TestRouterDiskSpill(t *testing.T) { testutils.SucceedsSoon(t, func() error { output.mu.Lock() spilled := output.mu.rowContainer.Spilled() - // nolint:deferunlock output.mu.Unlock() if !spilled { return errors.New("did not spill to disk") diff --git a/pkg/sql/schema_changer_test.go b/pkg/sql/schema_changer_test.go index f7b44f74345d..1ed2cca376e9 100644 --- a/pkg/sql/schema_changer_test.go +++ b/pkg/sql/schema_changer_test.go @@ -6629,7 +6629,6 @@ func TestCheckConstraintDropAndColumn(t *testing.T) { jobControlMu.Lock() scJob, err := s.JobRegistry().(*jobs.Registry).LoadJob(ctx, jobID) if err != nil { - // nolint:deferunlock jobControlMu.Unlock() return err } @@ -6675,7 +6674,6 @@ SET use_declarative_schema_changer = off; delayJobList = []string{"ALTER TABLE defaultdb.public.t ADD CHECK (i > 0)", "ALTER TABLE defaultdb.public.t DROP COLUMN j"} delayJobChannels = []chan struct{}{make(chan struct{}), make(chan struct{})} - // nolint:deferunlock jobControlMu.Unlock() go func() { diff --git a/pkg/sql/schemachanger/corpus/corpus.go b/pkg/sql/schemachanger/corpus/corpus.go index 5b54fc80ac97..386ff290111c 100644 --- a/pkg/sql/schemachanger/corpus/corpus.go +++ b/pkg/sql/schemachanger/corpus/corpus.go @@ -146,7 +146,6 @@ func (cc *Collector) GetBeforeStage( cc.mu.Lock() cc.corpusEntries = append(cc.corpusEntries, entry) cc.corpusPrefixes[corpusPrefix] = struct{}{} - // nolint:deferunlock cc.mu.Unlock() } return nil diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index 5fffe9cd3853..c0b172988521 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -6587,7 +6587,6 @@ SELECT // We do TryLock() as a best-effort guard against this, but it will be // racey. The caller is expected to have locked the mutex first. replicaMu.TryLock() - // nolint:deferunlock replicaMu.Unlock() } return tree.DBoolTrue, nil @@ -10063,7 +10062,6 @@ func GenerateUniqueInt(instanceID ProcessUniqueID) tree.DInt { timestamp = uniqueIntState.timestamp + 1 } uniqueIntState.timestamp = timestamp - // nolint:deferunlock uniqueIntState.Unlock() return GenerateUniqueID(int32(instanceID), timestamp) diff --git a/pkg/sql/sessiondata/sequence_state.go b/pkg/sql/sessiondata/sequence_state.go index 909d66fff379..e251d4b821ec 100644 --- a/pkg/sql/sessiondata/sequence_state.go +++ b/pkg/sql/sessiondata/sequence_state.go @@ -51,7 +51,6 @@ func (ss *SequenceState) RecordValue(seqID uint32, val int64) { ss.mu.Lock() ss.mu.lastSequenceIncremented = seqID ss.mu.latestValues[seqID] = val - // nolint:deferunlock ss.mu.Unlock() } @@ -60,7 +59,6 @@ func (ss *SequenceState) RecordValue(seqID uint32, val int64) { func (ss *SequenceState) SetLastSequenceIncremented(seqID uint32) { ss.mu.Lock() ss.mu.lastSequenceIncremented = seqID - // nolint:deferunlock ss.mu.Unlock() } diff --git a/pkg/sql/sqlliveness/slinstance/slinstance.go b/pkg/sql/sqlliveness/slinstance/slinstance.go index 3306958d90d8..f6b327063368 100644 --- a/pkg/sql/sqlliveness/slinstance/slinstance.go +++ b/pkg/sql/sqlliveness/slinstance/slinstance.go @@ -176,7 +176,6 @@ func (l *Instance) setSession(s *session) { // Notify all calls to Session that a non-nil session is available. close(l.mu.blockCh) l.mu.blockCh = nil - // nolint:deferunlock l.mu.Unlock() } @@ -354,7 +353,6 @@ func (l *Instance) heartbeatLoopInner(ctx context.Context) error { var s *session l.mu.Lock() s = l.mu.s - // nolint:deferunlock l.mu.Unlock() // If we don't currently have a session, create one. diff --git a/pkg/sql/sqlstats/ssmemstorage/ss_mem_iterator.go b/pkg/sql/sqlstats/ssmemstorage/ss_mem_iterator.go index 7ed32eb74e05..0da40cb84fbd 100644 --- a/pkg/sql/sqlstats/ssmemstorage/ss_mem_iterator.go +++ b/pkg/sql/sqlstats/ssmemstorage/ss_mem_iterator.go @@ -89,7 +89,6 @@ func (s *StmtStatsIterator) Next() bool { fullScan := statementStats.mu.fullScan database := statementStats.mu.database querySummary := statementStats.mu.querySummary - // nolint:deferunlock statementStats.mu.Unlock() s.currentValue = &appstatspb.CollectedStatementStatistics{ @@ -134,7 +133,6 @@ func NewTxnStatsIterator(container *Container, options sqlstats.IteratorOptions) for k := range container.mu.txns { txnKeys = append(txnKeys, k) } - // nolint:deferunlock container.mu.Unlock() if options.SortedKey { sort.Sort(txnKeys) diff --git a/pkg/sql/stmtdiagnostics/statement_diagnostics.go b/pkg/sql/stmtdiagnostics/statement_diagnostics.go index 17f68677159d..53fa814f66eb 100644 --- a/pkg/sql/stmtdiagnostics/statement_diagnostics.go +++ b/pkg/sql/stmtdiagnostics/statement_diagnostics.go @@ -668,7 +668,6 @@ func (r *Registry) pollRequests(ctx context.Context) error { for { r.mu.Lock() epoch := r.mu.epoch - // nolint:deferunlock r.mu.Unlock() var extraColumns string @@ -698,7 +697,6 @@ func (r *Registry) pollRequests(ctx context.Context) error { // manually while the query was running. In that case, if we were to process // the query results normally, we might remove that manually-added request. if r.mu.epoch != epoch { - // nolint:deferunlock r.mu.Unlock() continue } diff --git a/pkg/sql/tests/rsg_test.go b/pkg/sql/tests/rsg_test.go index 857f051fa175..93dd3f431b94 100644 --- a/pkg/sql/tests/rsg_test.go +++ b/pkg/sql/tests/rsg_test.go @@ -94,7 +94,6 @@ func (db *verifyFormatDB) Incr(sql string) func() { db.mu.active = make(map[string]int) } db.mu.active[sql]++ - // nolint:deferunlock db.mu.Unlock() return func() { @@ -104,7 +103,6 @@ func (db *verifyFormatDB) Incr(sql string) func() { if db.mu.active[sql] == 0 { delete(db.mu.active, sql) } - // nolint:deferunlock db.mu.Unlock() } } @@ -865,7 +863,6 @@ func testRandomSyntax( countsMu.total, countsMu.success, ) - // nolint:deferunlock countsMu.Unlock() } }(ctx) diff --git a/pkg/sql/type_change_test.go b/pkg/sql/type_change_test.go index b2e6a3a23200..47fdad6bd852 100644 --- a/pkg/sql/type_change_test.go +++ b/pkg/sql/type_change_test.go @@ -292,12 +292,10 @@ func TestEnumMemberTransitionIsolation(t *testing.T) { if blocker != nil { unblocker = blocker blocker = nil - // nolint:deferunlock mu.Unlock() <-unblocker return nil } - // nolint:deferunlock mu.Unlock() return errors.New("boom") }, @@ -331,11 +329,9 @@ CREATE TYPE ab AS ENUM ('a', 'b')`, for { mu.Lock() if blocker == nil { - // nolint:deferunlock mu.Unlock() break } - // nolint:deferunlock mu.Unlock() } _, err := sqlDB.Exec(`BEGIN; ALTER TYPE ab DROP VALUE 'b'; ALTER TYPE ab ADD VALUE 'd'; COMMIT`) diff --git a/pkg/storage/metamorphic/deck.go b/pkg/storage/metamorphic/deck.go index 68c1ef24f4e6..7ba1d29e25d0 100644 --- a/pkg/storage/metamorphic/deck.go +++ b/pkg/storage/metamorphic/deck.go @@ -63,7 +63,6 @@ func (d *deck) Int() int { } result := d.mu.deck[d.mu.index] d.mu.index++ - // nolint:deferunlock d.mu.Unlock() return result } diff --git a/pkg/storage/pebble.go b/pkg/storage/pebble.go index 8805b3691296..6a9d834bef63 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -1365,7 +1365,6 @@ func (p *Pebble) makeMetricEtcEventListener(ctx context.Context) pebble.EventLis } p.mu.Lock() cb := p.mu.flushCompletedCallback - // nolint:deferunlock p.mu.Unlock() if cb != nil { cb() @@ -1458,7 +1457,6 @@ func (p *Pebble) aggregateBatchCommitStats(stats BatchCommitStats) { p.batchCommitStats.L0ReadAmpWriteStallDuration += stats.L0ReadAmpWriteStallDuration p.batchCommitStats.WALRotationDuration += stats.WALRotationDuration p.batchCommitStats.CommitWaitDuration += stats.CommitWaitDuration - // nolint:deferunlock p.batchCommitStats.Unlock() } @@ -1917,11 +1915,9 @@ func (p *Pebble) GetMetrics() Metrics { } p.iterStats.Lock() m.Iterator = p.iterStats.AggregatedIteratorStats - // nolint:deferunlock p.iterStats.Unlock() p.batchCommitStats.Lock() m.BatchCommitStats = p.batchCommitStats.AggregatedBatchCommitStats - // nolint:deferunlock p.batchCommitStats.Unlock() return m } @@ -2197,7 +2193,6 @@ func (p *Pebble) CompactRange(start, end roachpb.Key) error { func (p *Pebble) RegisterFlushCompletedCallback(cb func()) { p.mu.Lock() p.mu.flushCompletedCallback = cb - // nolint:deferunlock p.mu.Unlock() } diff --git a/pkg/testutils/distsqlutils/row_buffer.go b/pkg/testutils/distsqlutils/row_buffer.go index 3d9a9c4b3815..cf16809dd6b5 100644 --- a/pkg/testutils/distsqlutils/row_buffer.go +++ b/pkg/testutils/distsqlutils/row_buffer.go @@ -141,7 +141,6 @@ func (rb *RowBuffer) Push( func (rb *RowBuffer) ProducerClosed() bool { rb.Mu.Lock() c := rb.Mu.producerClosed - // nolint:deferunlock rb.Mu.Unlock() return c } diff --git a/pkg/testutils/lint/lint_test.go b/pkg/testutils/lint/lint_test.go index 4f4b47e9f2ab..fe08bbf22ef0 100644 --- a/pkg/testutils/lint/lint_test.go +++ b/pkg/testutils/lint/lint_test.go @@ -563,6 +563,7 @@ func TestLint(t *testing.T) { ":!*/doc.go", ":!util/syncutil/mutex_sync.go", ":!util/syncutil/mutex_sync_race.go", + ":!testutils/lint/passes/deferunlockcheck/testdata/src/github.com/cockroachdb/cockroach/pkg/util/syncutil/mutex_sync.go", ) if err != nil { t.Fatal(err) diff --git a/pkg/testutils/lint/passes/deferunlockcheck/BUILD.bazel b/pkg/testutils/lint/passes/deferunlockcheck/BUILD.bazel index 96831507e7ff..16b203b95d71 100644 --- a/pkg/testutils/lint/passes/deferunlockcheck/BUILD.bazel +++ b/pkg/testutils/lint/passes/deferunlockcheck/BUILD.bazel @@ -6,10 +6,12 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/deferunlockcheck", visibility = ["//visibility:public"], deps = [ + "//pkg/testutils/lint/passes/loopvarcapture", "//pkg/testutils/lint/passes/passesutil", "@org_golang_x_tools//go/analysis", "@org_golang_x_tools//go/analysis/passes/inspect", "@org_golang_x_tools//go/ast/inspector", + "@org_golang_x_tools//go/types/typeutil", ], ) diff --git a/pkg/testutils/lint/passes/deferunlockcheck/deferunlockcheck.go b/pkg/testutils/lint/passes/deferunlockcheck/deferunlockcheck.go index 4b04889d32f7..bfdedcab8028 100644 --- a/pkg/testutils/lint/passes/deferunlockcheck/deferunlockcheck.go +++ b/pkg/testutils/lint/passes/deferunlockcheck/deferunlockcheck.go @@ -11,18 +11,25 @@ package deferunlockcheck import ( + "fmt" "go/ast" + "go/token" + "go/types" + "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/loopvarcapture" "github.com/cockroachdb/cockroach/pkg/testutils/lint/passes/passesutil" "golang.org/x/tools/go/analysis" "golang.org/x/tools/go/analysis/passes/inspect" "golang.org/x/tools/go/ast/inspector" + "golang.org/x/tools/go/types/typeutil" ) -// Doc documents this pass. -const Doc = "checks that usages of mutex Unlock() are deferred." - -const noLintName = "deferunlock" +const ( + // Doc documents this pass. + Doc = "checks that usages of mutex Unlock() are deferred." + noLintName = "deferunlock" + maxLineDistance = 5 +) // Analyzer is an analysis pass that checks for mutex unlocks which // aren't deferred. @@ -33,27 +40,277 @@ var Analyzer = &analysis.Analyzer{ Run: run, } +var mutexFunctions = []loopvarcapture.Function{ + {Pkg: "github.com/cockroachdb/cockroach/pkg/util/syncutil", Type: "Mutex", Name: "Lock"}, + {Pkg: "github.com/cockroachdb/cockroach/pkg/util/syncutil", Type: "Mutex", Name: "Unlock"}, + {Pkg: "github.com/cockroachdb/cockroach/pkg/util/syncutil", Type: "RWMutex", Name: "Lock"}, + {Pkg: "github.com/cockroachdb/cockroach/pkg/util/syncutil", Type: "RWMutex", Name: "RLock"}, + {Pkg: "github.com/cockroachdb/cockroach/pkg/util/syncutil", Type: "RWMutex", Name: "Unlock"}, + {Pkg: "github.com/cockroachdb/cockroach/pkg/util/syncutil", Type: "RWMutex", Name: "RUnlock"}, + {Pkg: "sync", Type: "Mutex", Name: "Lock"}, + {Pkg: "sync", Type: "Mutex", Name: "Unlock"}, + {Pkg: "sync", Type: "RWMutex", Name: "Lock"}, + {Pkg: "sync", Type: "RWMutex", Name: "Unlock"}, + {Pkg: "sync", Type: "RWMutex", Name: "RLock"}, + {Pkg: "sync", Type: "RWMutex", Name: "RUnlock"}, +} + +type LockTracker struct { + body *ast.BlockStmt + pass *analysis.Pass + locks []*LockInfo + issues []analysis.Diagnostic +} + +type LockInfo struct { + // the prefix structure before calling Lock()/Unlock() e.g. s.mu. + prefix string + // lineNum tells us where the lock/unlock statement is. + lineNum int + // isRead tells us if its a read lock or normal lock + isRead bool + // foundUnlockMatch tells us if we found an unlock for this lock. Used to determine + // if we should report an issue when we find an if condition, loop or function call + // and haven't found a matching unlock. + foundUnlockMatch bool +} + func run(pass *analysis.Pass) (interface{}, error) { astInspector := pass.ResultOf[inspect.Analyzer].(*inspector.Inspector) // Note: defer ...Unlock() expressions are captured as ast.DeferStmt nodes so we don't have to worry about those results returning // for ast.ExprStmt nodes. filter := []ast.Node{ - (*ast.ExprStmt)(nil), + (*ast.FuncDecl)(nil), } astInspector.Preorder(filter, func(n ast.Node) { - stmt := n.(*ast.ExprStmt) - expr, ok := stmt.X.(*ast.CallExpr) - if !ok { + fn := n.(*ast.FuncDecl) + if fn == nil || fn.Body == nil { return } - sel, ok := expr.Fun.(*ast.SelectorExpr) + lockTracker := NewLockTracker(fn.Body, pass) + for _, issue := range lockTracker.findIssues() { + pass.Report(issue) + } + }) + + return nil, nil +} + +func NewLockTracker(body *ast.BlockStmt, pass *analysis.Pass) *LockTracker { + return &LockTracker{ + body: body, + pass: pass, + locks: []*LockInfo{}, + issues: []analysis.Diagnostic{}, + } +} + +func (lt *LockTracker) findIssues() []analysis.Diagnostic { + ast.Inspect(lt.body, lt.traverseBody) + return lt.issues +} + +func (lt *LockTracker) traverseBody(n ast.Node) bool { + switch node := n.(type) { + // This case is where we find Locks and Unlocks. If we don't find them, + // This is some other function call and we should see if there are any + // locks without a matching lock. + // We also are only done if this happens to be a Lock/Unlock otherwise + // it could be an inline function and we need to keep going. + case *ast.CallExpr: + if lt.isMutexCall(node) { + return false + } else if lt.shouldReportNonLinearControlFlow(node) { + lt.addNonLinearControlFlowIssue(node.Pos(), "function call") + + } + // Check if this if statement is after a Lock that doesn't have a matching + // Unlock. + case *ast.IfStmt: + if lt.shouldReportNonLinearControlFlow(node) { + lt.addNonLinearControlFlowIssue(node.Body.Pos(), "if statement") + return false + } + // Check if this loop is after a Lock that doesn't have a matching Unlock. + case *ast.RangeStmt, *ast.ForStmt: + if lt.shouldReportNonLinearControlFlow(node) { + lt.addNonLinearControlFlowIssue(node.Pos(), "for loop") + return false + } + case *ast.DeferStmt: + // We only want to stop traversal if we found a defer Unlock. + // If it is a deferred function we want to keep traversing the body + // in order to see if there is an Unlock inside. + if lt.isMutexCall(node.Call) { + return false + } + fn, ok := node.Call.Fun.(*ast.FuncLit) if !ok { - return + return false } - if sel.Sel != nil && (sel.Sel.Name == "Unlock" || sel.Sel.Name == "RUnlock") && !passesutil.HasNolintComment(pass, n, noLintName) { - pass.Reportf(sel.Pos(), "Mutex %s not deferred", sel.Sel.Name) + ast.Inspect(fn.Body, lt.traverseBody) + return false + } + // Keep traversing if we didn't find any relevant nodes. + return true +} + +// isMutexCall determines where the given *ast.CallExpr is a Lock/Unlock. +// If it is a Lock/Unlock it will return true but will first: +// - If it is a Lock it calls addLock +// - If it is an Unlock it calls maybeReportUnlock. +// +// Otherewise returning false. +func (lt *LockTracker) isMutexCall(call *ast.CallExpr) bool { + callee, ok := typeutil.Callee(lt.pass.TypesInfo, call).(*types.Func) + if !ok { + return false + } + pkg := callee.Pkg() + if pkg == nil { + return false + } + calleePkg := pkg.Path() + calleeFunc := callee.Name() + calleeObj := "" + + recv := callee.Type().(*types.Signature).Recv() + if recv != nil { + // if there is a receiver (i.e., this is a method call), get the + // name of the type of the receiver + recvType := recv.Type() + if pointerType, ok := recvType.(*types.Pointer); ok { + recvType = pointerType.Elem() } + named, ok := recvType.(*types.Named) + if !ok { + return false + } + + calleeObj = named.Obj().Name() + } + for _, fn := range mutexFunctions { + if fn.Pkg == calleePkg && fn.Type == calleeObj && fn.Name == calleeFunc { + switch calleeFunc { + case "Lock": + lt.addLock(call, false) + case "RLock": + lt.addLock(call, true) + case "Unlock": + lt.maybeReportUnlock(call, false) + case "RUnlock": + lt.maybeReportUnlock(call, true) + } + return true + } + } + + return false +} + +// addLock adds a new LockInfo to the end of the locks slice. +func (lt *LockTracker) addLock(call *ast.CallExpr, isRead bool) { + sel, ok := call.Fun.(*ast.SelectorExpr) + if !ok { + return + } + // We add the other fields when we traverse and reach the *ast.Ident + lt.locks = append(lt.locks, &LockInfo{ + isRead: isRead, + foundUnlockMatch: false, + }) + ast.Inspect(sel, func(n ast.Node) bool { + ident, ok := n.(*ast.Ident) + if !ok { + return true + } + // If we find Lock or RLock set the line and we are done + // traversing. + if ident.Name == "Lock" || ident.Name == "RLock" { + position := lt.pass.Fset.Position(n.Pos()) + lt.locks[len(lt.locks)-1].lineNum = position.Line + return false + } + // Otherwise its part of the prefix that we'll add. + lt.locks[len(lt.locks)-1].prefix += ident.Name + "." + // Keep traversing if we didn't find an *ast.Ident node. + return true }) +} - return nil, nil +// maybeReportUnlock tries to find a matching lock for a given unlock by +// iterating backwards in the locks slice. If one is found, it checks if the +// distance between is greater than maxLineDistance and also has no nlint +// comment and reports on that if both are true. +func (lt *LockTracker) maybeReportUnlock(call *ast.CallExpr, isRead bool) { + sel, ok := call.Fun.(*ast.SelectorExpr) + if !ok { + return + } + var node ast.Node + var unlockPrefix string + var unlockLineNum int + var unlockPos token.Pos + ast.Inspect(sel, func(n ast.Node) bool { + ident, ok := n.(*ast.Ident) + if !ok { + return true + } + // If we find Unlock or RUnlock set the line and we are done + // traversing. + if ident.Name == "Unlock" || ident.Name == "RUnlock" { + position := lt.pass.Fset.Position(n.Pos()) + unlockPos = n.Pos() + unlockLineNum = position.Line + node = n + return false + } + // Otherwise its part of the prefix that we'll add. + unlockPrefix += ident.Name + "." + // Keep traversing if we didn't find an *ast.Ident node. + return true + }) + // Reverse iterate through the locks slice searching for a match. + for i := len(lt.locks) - 1; i >= 0; i-- { + lock := lt.locks[i] + if lock.foundUnlockMatch { + continue + } + // It's only an issue if prefixes match, the lock type matches, distance between is >5 lines away and + // there is no nolint comment. + if unlockPrefix == lock.prefix && isRead == lock.isRead { + lockDistance := unlockLineNum - lock.lineNum + if lockDistance > maxLineDistance && !passesutil.HasNolintComment(lt.pass, node, noLintName) { + lt.issues = append(lt.issues, analysis.Diagnostic{ + Pos: unlockPos, + Message: fmt.Sprintf("Unlock is >%d lines away from matching Lock, move it to a defer statement after Lock", maxLineDistance), + }) + } + // This should set on close enough unlocks, nolint unlocks and defer unlocks. + lock.foundUnlockMatch = true + } + } +} + +// shouldReportNonLinearControlFlow checks if locks has a *LockInfo without a matching unlock +// and if the node has a nolint comment near it. If it finds both it returns true. +func (lt *LockTracker) shouldReportNonLinearControlFlow(node ast.Node) bool { + hasNoLintComment := passesutil.HasNolintComment(lt.pass, node, noLintName) + if hasNoLintComment { + return false + } + for i := len(lt.locks) - 1; i >= 0; i-- { + lock := lt.locks[i] + if !lock.foundUnlockMatch && !hasNoLintComment { + return true + } + } + return false +} + +func (lt *LockTracker) addNonLinearControlFlowIssue(pos token.Pos, stmt string) { + lt.issues = append(lt.issues, analysis.Diagnostic{ + Pos: pos, + Message: fmt.Sprintf("%s between Lock and Unlock may be unsafe, move Unlock to a defer statement after Lock", stmt), + }) } diff --git a/pkg/testutils/lint/passes/deferunlockcheck/testdata/src/a/a.go b/pkg/testutils/lint/passes/deferunlockcheck/testdata/src/a/a.go index 0297007d25d7..fdf43a5c8e80 100644 --- a/pkg/testutils/lint/passes/deferunlockcheck/testdata/src/a/a.go +++ b/pkg/testutils/lint/passes/deferunlockcheck/testdata/src/a/a.go @@ -17,42 +17,150 @@ import ( type TestUnlockLint struct { mu struct { syncutil.Mutex + foo string + too bool } + amended bool } -func init() { - var mut syncutil.Mutex - var rwmut syncutil.RWMutex - testUnlock := &TestUnlockLint{} - - // Test the main use case. - // Should only capture Unlock() - testUnlock.mu.Lock() - testUnlock.mu.Unlock() // want `Mutex Unlock not deferred` - // This should pass. - defer testUnlock.mu.Unlock() - - // Test within a function. - okFn := func() { - testUnlock.mu.Lock() - defer testUnlock.mu.Unlock() +func testFnCall() bool { + return true +} + +func basicCases() { + t := &TestUnlockLint{} + + // Less than or equal to 5 lines. + t.mu.Lock() + t.mu.foo = "a" + t.mu.Unlock() + + // Unlock is too far away. + t.mu.Lock() + // Lines + // Lines + // Lines + // Lines + // Lines + t.mu.Unlock() // want `Unlock is >5 lines away from matching Lock, move it to a defer statement after Lock` + + // nolint exception inline. + t.mu.Lock() + // Lines + // Lines + // Lines + // Lines + // Lines + t.mu.Unlock() // nolint:deferunlock + + // nolint exception above. + t.mu.Lock() + // Lines + // Lines + // Lines + // Lines + // Lines + // nolint:deferunlock + t.mu.Unlock() + + // If statement between lock/unlock pair. + someVar := "foo" + t.mu.Lock() + if someVar != "" { // want `if statement between Lock and Unlock may be unsafe, move Unlock to a defer statement after Lock` + t.mu.foo = "bar" + } + t.mu.Unlock() + + // For statement between lock/unlock pair. + someArr := []string{ + "a", + "b", + "c", + } + t.mu.Lock() + for i := len(someArr); i < len(someArr); i++ { // want `for loop between Lock and Unlock may be unsafe, move Unlock to a defer statement after Lock` + t.mu.foo = someArr[i] } - failFn := func() { - testUnlock.mu.Lock() - testUnlock.mu.Unlock() // want `Mutex Unlock not deferred` + t.mu.Unlock() + + // Range statement between lock/unlock pair. + t.mu.Lock() + for _, v := range someArr { // want `for loop between Lock and Unlock may be unsafe, move Unlock to a defer statement after Lock` + t.mu.foo = v } - okFn() - failFn() + t.mu.Unlock() - // Test mut variation. - defer mut.Unlock() - mut.Unlock() // want `Mutex Unlock not deferred` + // Function calls between lock/unlock pair. + t.mu.Lock() + testFnCall() // want `function call between Lock and Unlock may be unsafe, move Unlock to a defer statement after Lock` + t.mu.Unlock() - // Test RUnlock - defer rwmut.RUnlock() - rwmut.RUnlock() // want `Mutex RUnlock not deferred` + t.mu.Lock() + t.mu.too = testFnCall() // want `function call between Lock and Unlock may be unsafe, move Unlock to a defer statement after Lock` + t.mu.Unlock() - // Test the no lint rule. + // Allow non linear control flow if the nolint comment is present. + t.mu.Lock() // nolint:deferunlock - testUnlock.mu.Unlock() + if someVar != "" { + t.mu.foo = "bar" + } + t.mu.Unlock() + + t.mu.Lock() + for _, v := range someArr { // nolint:deferunlock + t.mu.foo = v + } + t.mu.Unlock() + + t.mu.Lock() + testFnCall() // nolint:deferunlock + t.mu.Unlock() + + // Allow deferring unlocks. + t.mu.Lock() + defer t.mu.Unlock() +} + +func deferFunc() { + t := &TestUnlockLint{} + // Having an unlock in a defer function w/out non linear control flow is ok. + t.mu.Lock() + defer func() { + t.mu.Unlock() + }() + + // Still catches if Unlock is too far. + t.mu.Lock() + defer func() { + // Lines + // Lines + // Lines + // Lines + // Lines + t.mu.Unlock() // want `Unlock is >5 lines away from matching Lock, move it to a defer statement after Lock` + }() +} + +func inlineFunc() { + t := &TestUnlockLint{} + // Inline function with deferred unlock is ok. + func() { + t.mu.Lock() + defer t.mu.Unlock() + t.mu.foo = "a" + t.mu.too = true + }() + // Same with inline function and no non linear control flow in between. + func() { + t.mu.Lock() + t.mu.foo = "a" + t.mu.Unlock() + }() + // This will produce a non linear control flow error. + func() { + t.mu.Lock() + testFnCall() // want `function call between Lock and Unlock may be unsafe, move Unlock to a defer statement after Lock` + t.mu.Unlock() + }() } diff --git a/pkg/testutils/net.go b/pkg/testutils/net.go index a0cff35e0f28..c88b95904c59 100644 --- a/pkg/testutils/net.go +++ b/pkg/testutils/net.go @@ -207,7 +207,6 @@ func NewPartitionableConn(serverConn net.Conn) *PartitionableConn { }) c.mu.Lock() c.mu.err = err - // nolint:deferunlock c.mu.Unlock() if err := c.clientConn.Close(); err != nil { log.Errorf(context.TODO(), "unexpected error closing internal pipe: %s", err) @@ -230,7 +229,6 @@ func NewPartitionableConn(serverConn net.Conn) *PartitionableConn { }) c.mu.Lock() c.mu.err = err - // nolint:deferunlock c.mu.Unlock() if err := c.clientConn.Close(); err != nil { log.Fatalf(context.TODO(), "unexpected error closing internal pipe: %s", err) @@ -306,7 +304,6 @@ func (c *PartitionableConn) UnpartitionS2C() { func (c *PartitionableConn) Read(b []byte) (n int, err error) { c.mu.Lock() err = c.mu.err - // nolint:deferunlock c.mu.Unlock() if err != nil { return 0, err @@ -320,7 +317,6 @@ func (c *PartitionableConn) Read(b []byte) (n int, err error) { func (c *PartitionableConn) Write(b []byte) (n int, err error) { c.mu.Lock() err = c.mu.err - // nolint:deferunlock c.mu.Unlock() if err != nil { return 0, err @@ -374,7 +370,6 @@ func (c *PartitionableConn) copyFromBuffer( src.Mutex.Lock() waitForNoPartitionLocked() data, err := src.readLocked(1024 * 1024) - // nolint:deferunlock src.Mutex.Unlock() if len(data) > 0 { diff --git a/pkg/testutils/storageutils/mocking.go b/pkg/testutils/storageutils/mocking.go index 4e6b883edee0..4de75fb0b3e4 100644 --- a/pkg/testutils/storageutils/mocking.go +++ b/pkg/testutils/storageutils/mocking.go @@ -73,7 +73,6 @@ func (c *ReplayProtectionFilterWrapper) run(args kvserverbase.FilterArgs) *kvpb. c.Lock() if pErr, ok := c.processedCommands[mapKey]; ok { - // nolint:deferunlock c.Unlock() return shallowCloneErrorWithTxn(pErr) } diff --git a/pkg/testutils/testfixtures/test_fixtures.go b/pkg/testutils/testfixtures/test_fixtures.go index 6d07a3a79af8..e206846a65ab 100644 --- a/pkg/testutils/testfixtures/test_fixtures.go +++ b/pkg/testutils/testfixtures/test_fixtures.go @@ -56,11 +56,9 @@ func ReuseOrGenerate(tb testing.TB, name string, generate func(dir string)) (dir ch, ok := mu.inProgress[name] if !ok { mu.inProgress[name] = make(chan struct{}) - // nolint:deferunlock mu.Unlock() break } - // nolint:deferunlock mu.Unlock() if first { tb.Logf("waiting for fixture %q", name) diff --git a/pkg/util/admission/work_queue.go b/pkg/util/admission/work_queue.go index fc75b633ef90..98da8a399e18 100644 --- a/pkg/util/admission/work_queue.go +++ b/pkg/util/admission/work_queue.go @@ -471,7 +471,6 @@ func (q *WorkQueue) startClosingEpochs() { q.mu.Lock() q.sampleEpochLIFOSettingsLocked() nextCloseTime := q.nextEpochCloseTimeLocked() - // nolint:deferunlock q.mu.Unlock() timeNow := q.timeNow() timerDur := nextCloseTime.Sub(timeNow) @@ -845,12 +844,10 @@ func (q *WorkQueue) granted(grantChainID grantChainID) int64 { now := q.timeNow() q.mu.Lock() if len(q.mu.tenantHeap) == 0 { - // nolint:deferunlock q.mu.Unlock() return 0 } if fn := q.knobs.DisableWorkQueueGranting; fn != nil && fn() { - // nolint:deferunlock q.mu.Unlock() return 0 } @@ -1878,7 +1875,6 @@ func (q *StoreWorkQueue) Admit( // from StoreWorkQueue.Admit(). q.mu.RLock() info.RequestedCount = q.mu.estimates.writeTokens - // nolint:deferunlock q.mu.RUnlock() } if info.ReplicatedWorkInfo.Enabled { @@ -1995,7 +1991,6 @@ func (q *StoreWorkQueue) admittedReplicatedWork( } additionalTokensNeeded := q.granters[wc].storeReplicatedWorkAdmittedLocked(originalTokens, replicatedWorkAdmittedInfo) if !coordMuLocked { - // nolint:deferunlock q.coordMu.Unlock() } q.q[wc].adjustTenantUsed(tenantID, additionalTokensNeeded) @@ -2063,7 +2058,6 @@ func (q *StoreWorkQueue) StatsToIgnore(ingestStats pebble.IngestOperationStats) q.mu.Lock() q.mu.stats.statsToIgnore.Bytes += ingestStats.Bytes q.mu.stats.statsToIgnore.ApproxIngestedIntoL0Bytes += ingestStats.ApproxIngestedIntoL0Bytes - // nolint:deferunlock q.mu.Unlock() } @@ -2079,7 +2073,6 @@ func (q *StoreWorkQueue) updateStoreStatsAfterWorkDone( q.mu.stats.aux.writeBypassedAccountedBytes += uint64(doneInfo.WriteBytes) q.mu.stats.aux.ingestedBypassedAccountedBytes += uint64(doneInfo.IngestedBytes) } - // nolint:deferunlock q.mu.Unlock() } @@ -2115,7 +2108,6 @@ func (q *StoreWorkQueue) getStoreAdmissionStats() storeAdmissionStats { func (q *StoreWorkQueue) setStoreRequestEstimates(estimates storeRequestEstimates) { q.mu.Lock() q.mu.estimates = estimates - // nolint:deferunlock q.mu.Unlock() } @@ -2192,7 +2184,6 @@ func (q *StoreWorkQueue) sequenceReplicatedWork(createTime int64, info Replicate seq = &sequencer{} q.sequencersMu.s[info.RangeID] = seq } - // nolint:deferunlock q.sequencersMu.Unlock() // We're assuming sequenceReplicatedWork is never invoked concurrently for a // given RangeID. diff --git a/pkg/util/admission/work_queue_test.go b/pkg/util/admission/work_queue_test.go index 08bf1419e076..1c30b9c5a3f5 100644 --- a/pkg/util/admission/work_queue_test.go +++ b/pkg/util/admission/work_queue_test.go @@ -424,7 +424,6 @@ func TestWorkQueueTokenResetRace(t *testing.T) { q.close() mu.Lock() t.Logf("total: %d, err: %d", totalCount, errCount) - // nolint:deferunlock mu.Unlock() } diff --git a/pkg/util/circuit/circuitbreaker.go b/pkg/util/circuit/circuitbreaker.go index a7f4668a8342..bdd10dd50de4 100644 --- a/pkg/util/circuit/circuitbreaker.go +++ b/pkg/util/circuit/circuitbreaker.go @@ -141,7 +141,6 @@ func (b *Breaker) Report(err error) { // We get to write the error since we have exclusive access via b.mu. b.mu.errAndCh.err = storeErr close(b.mu.errAndCh.ch) - // nolint:deferunlock b.mu.Unlock() opts := b.Opts() @@ -234,7 +233,6 @@ func TestingSetTripped(b *Breaker, err error) (undo func()) { func (b *Breaker) maybeTriggerProbe(force bool) { b.mu.Lock() if b.mu.probing || (!force && b.mu.errAndCh.err == nil) { - // nolint:deferunlock b.mu.Unlock() // A probe is already running or the breaker is not currently tripped. The // latter case can occur since maybeTriggerProbe is invoked from diff --git a/pkg/util/every_n.go b/pkg/util/every_n.go index e057214d2a46..4d1fbb6a5bc4 100644 --- a/pkg/util/every_n.go +++ b/pkg/util/every_n.go @@ -48,7 +48,6 @@ func (e *EveryN) ShouldProcess(now time.Time) bool { shouldProcess = true e.lastProcessed = now } - // nolint:deferunlock e.Unlock() return shouldProcess } diff --git a/pkg/util/goschedstats/runnable.go b/pkg/util/goschedstats/runnable.go index 0abfa28741e5..3f1a60fed778 100644 --- a/pkg/util/goschedstats/runnable.go +++ b/pkg/util/goschedstats/runnable.go @@ -165,7 +165,6 @@ func init() { t := <-ticker.C callbackInfo.mu.Lock() cbs := callbackInfo.cbs - // nolint:deferunlock callbackInfo.mu.Unlock() sst.getStatsOnTick(t, cbs, ticker) } diff --git a/pkg/util/hlc/hlc.go b/pkg/util/hlc/hlc.go index 32628c400737..bfb3980dd178 100644 --- a/pkg/util/hlc/hlc.go +++ b/pkg/util/hlc/hlc.go @@ -138,7 +138,6 @@ func (m *HybridManualClock) UnixNano() int64 { m.mu.RLock() nanosAtPause := m.mu.nanosAtPause nanos := m.mu.nanos - // nolint:deferunlock m.mu.RUnlock() if nanosAtPause > 0 { return nanos + nanosAtPause @@ -150,7 +149,6 @@ func (m *HybridManualClock) UnixNano() int64 { func (m *HybridManualClock) Increment(nanos int64) { m.mu.Lock() m.mu.nanos += nanos - // nolint:deferunlock m.mu.Unlock() } @@ -175,7 +173,6 @@ func (m *HybridManualClock) Forward(tsNanos int64) { func (m *HybridManualClock) Pause() { m.mu.Lock() m.mu.nanosAtPause = timeutil.Now().UnixNano() - // nolint:deferunlock m.mu.Unlock() } @@ -184,7 +181,6 @@ func (m *HybridManualClock) Pause() { func (m *HybridManualClock) Resume() { m.mu.Lock() m.mu.nanosAtPause = 0 - // nolint:deferunlock m.mu.Unlock() } diff --git a/pkg/util/interval/btree_based_interval.go b/pkg/util/interval/btree_based_interval.go index a516781678ce..a60fad9b8ccf 100644 --- a/pkg/util/interval/btree_based_interval.go +++ b/pkg/util/interval/btree_based_interval.go @@ -51,14 +51,12 @@ func (f *FreeList) newNode() (n *node) { f.mu.Lock() index := len(f.freelist) - 1 if index < 0 { - // nolint:deferunlock f.mu.Unlock() return new(node) } n = f.freelist[index] f.freelist[index] = nil f.freelist = f.freelist[:index] - // nolint:deferunlock f.mu.Unlock() return } @@ -71,7 +69,6 @@ func (f *FreeList) freeNode(n *node) (out bool) { f.freelist = append(f.freelist, n) out = true } - // nolint:deferunlock f.mu.Unlock() return } diff --git a/pkg/util/log/buffered_sink.go b/pkg/util/log/buffered_sink.go index 7dccf10ab931..84079b8fbb45 100644 --- a/pkg/util/log/buffered_sink.go +++ b/pkg/util/log/buffered_sink.go @@ -358,6 +358,12 @@ func (bs *bufferedSink) runFlusher(stopC <-chan struct{}) { return logging.mu.exitOverride.f }() code := bs.exitCode() + if f != nil { + f(code, err) + } else { + exit.WithCode(code) + } + } } if done { return diff --git a/pkg/util/log/buffered_sink_closer_test.go b/pkg/util/log/buffered_sink_closer_test.go index 48d090e5020d..8f57a5afc88b 100644 --- a/pkg/util/log/buffered_sink_closer_test.go +++ b/pkg/util/log/buffered_sink_closer_test.go @@ -72,14 +72,12 @@ func TestBufferSinkDone(t *testing.T) { closer.mu.Lock() assert.Len(t, closer.mu.sinkRegistry, 1, "expected sink registry to include registered bufferedSink") - // nolint:deferunlock closer.mu.Unlock() closer.bufferedSinkDone(bs) closer.mu.Lock() assert.Empty(t, closer.mu.sinkRegistry, "expected sink registry to be empty") - // nolint:deferunlock closer.mu.Unlock() require.NoError(t, closer.Close(time.Second /* timeout */), "bufferedSinkCloser timed out unexpectedly") @@ -96,7 +94,6 @@ func TestBufferSinkDone(t *testing.T) { _, ok := closer.mu.sinkRegistry[bs1] assert.Len(t, closer.mu.sinkRegistry, 1, "length of bufferSink registry larger than expected") assert.True(t, ok, "expected bufferSink to be in registry") - // nolint:deferunlock closer.mu.Unlock() require.Panics(t, func() { diff --git a/pkg/util/log/channels.go b/pkg/util/log/channels.go index e11bc2a3f707..a084bcc88a22 100644 --- a/pkg/util/log/channels.go +++ b/pkg/util/log/channels.go @@ -64,7 +64,6 @@ func logfDepthInternal( if logging.mu.exitOverride.f != nil { exitFunc = logging.mu.exitOverride.f } - // nolint:deferunlock logging.mu.Unlock() // Fatal error handling later already tries to exit even if I/O should diff --git a/pkg/util/log/channels_test.go b/pkg/util/log/channels_test.go index 7113439bf7d8..420ad746fa9d 100644 --- a/pkg/util/log/channels_test.go +++ b/pkg/util/log/channels_test.go @@ -99,7 +99,6 @@ func TestRepro81025(t *testing.T) { // our blocking sink. logging.rmu.Lock() logging.rmu.currentStderrSinkInfo.sink = s - // nolint:deferunlock logging.rmu.Unlock() // Hijack the exit func to ensure we don't exit during the test when seeing a @@ -110,7 +109,6 @@ func TestRepro81025(t *testing.T) { defer exitCode.mu.Unlock() exitCode.mu.observedExitCode = &code } - // nolint:deferunlock logging.mu.Unlock() // Force the first log file into existence so that we can wrap its buffer in @@ -125,7 +123,6 @@ func TestRepro81025(t *testing.T) { doneC: make(chan struct{}), } s.mu.file = sb - // nolint:deferunlock s.mu.Unlock() listFiles := func() ([]string, error) { @@ -201,7 +198,6 @@ func TestRepro81025(t *testing.T) { // The fatal error eventually came through after the rotation was unblocked. exitCode.mu.Lock() require.Equal(t, exit.FatalError(), *exitCode.mu.observedExitCode) - // nolint:deferunlock exitCode.mu.Unlock() // Three files now, as the first one was rotated. diff --git a/pkg/util/log/clog.go b/pkg/util/log/clog.go index 6b88764a3741..085b7911cf8b 100644 --- a/pkg/util/log/clog.go +++ b/pkg/util/log/clog.go @@ -304,7 +304,6 @@ func (l *loggerT) outputLogEntry(entry logEntry) { } exitFunc = logging.mu.exitOverride.f } - // nolint:deferunlock logging.mu.Unlock() exitCalled := make(chan struct{}) diff --git a/pkg/util/log/clog_test.go b/pkg/util/log/clog_test.go index eca5ca444e4b..f6af4b89bf3f 100644 --- a/pkg/util/log/clog_test.go +++ b/pkg/util/log/clog_test.go @@ -78,12 +78,10 @@ func capture() func() { fileSink.mu.Lock() oldFile := fileSink.mu.file fileSink.mu.file = new(flushBuffer) - // nolint:deferunlock fileSink.mu.Unlock() return func() { fileSink.mu.Lock() fileSink.mu.file = oldFile - // nolint:deferunlock fileSink.mu.Unlock() } } @@ -698,7 +696,6 @@ func TestExitOnFullDisk(t *testing.T) { l.outputMu.Lock() l.exitLocked(fmt.Errorf("out of space"), exit.UnspecifiedError()) - // nolint:deferunlock l.outputMu.Unlock() exited.Wait() diff --git a/pkg/util/log/exit_override.go b/pkg/util/log/exit_override.go index 84f1c5a3396a..9d9df43b4c31 100644 --- a/pkg/util/log/exit_override.go +++ b/pkg/util/log/exit_override.go @@ -32,7 +32,6 @@ var makeProcessUnavailableFunc struct { func MakeProcessUnavailable() { makeProcessUnavailableFunc.Lock() fn := makeProcessUnavailableFunc.fn - // nolint:deferunlock makeProcessUnavailableFunc.Unlock() if fn != nil { fn() @@ -44,7 +43,6 @@ func MakeProcessUnavailable() { func SetMakeProcessUnavailableFunc(fn func()) { makeProcessUnavailableFunc.Lock() makeProcessUnavailableFunc.fn = fn - // nolint:deferunlock makeProcessUnavailableFunc.Unlock() } @@ -92,7 +90,6 @@ func (l *loggerT) exitLocked(err error, code exit.Code) { logging.mu.Lock() f := logging.mu.exitOverride.f - // nolint:deferunlock logging.mu.Unlock() if f != nil { f(code, err) diff --git a/pkg/util/log/file.go b/pkg/util/log/file.go index 500c15423b8c..52f0ed218c64 100644 --- a/pkg/util/log/file.go +++ b/pkg/util/log/file.go @@ -167,7 +167,6 @@ func (l *fileSink) attachHints(stacks []byte) []byte { l.mu.Lock() stacks = append(stacks, []byte(fmt.Sprintf( "\nFor more context, check log files in: %s\n", l.mu.logDir))...) - // nolint:deferunlock l.mu.Unlock() return stacks } diff --git a/pkg/util/log/file_api.go b/pkg/util/log/file_api.go index cb17b44eb066..cf2b1d477ba4 100644 --- a/pkg/util/log/file_api.go +++ b/pkg/util/log/file_api.go @@ -131,7 +131,6 @@ func ListLogFiles() (logFiles []logpb.FileInfo, err error) { err = logging.allSinkInfos.iterFileSinks(func(l *fileSink) error { l.mu.Lock() thisLogDir := l.mu.logDir - // nolint:deferunlock l.mu.Unlock() if !l.enabled.Get() || thisLogDir == "" { // This file sink is detached from file storage. @@ -156,7 +155,6 @@ func (l *fileSink) listLogFiles() (string, []logpb.FileInfo, error) { var results []logpb.FileInfo l.mu.Lock() dir := l.mu.logDir - // nolint:deferunlock l.mu.Unlock() if dir == "" { // No log directory configured: simply indicate that there are no diff --git a/pkg/util/log/file_log_gc.go b/pkg/util/log/file_log_gc.go index 5f88de7a2a3f..248d1ee55619 100644 --- a/pkg/util/log/file_log_gc.go +++ b/pkg/util/log/file_log_gc.go @@ -30,7 +30,6 @@ func (l *fileSink) gcDaemon(ctx context.Context) { logging.mu.Lock() doGC := !logging.mu.disableDaemons - // nolint:deferunlock logging.mu.Unlock() if doGC { diff --git a/pkg/util/log/file_log_gc_test.go b/pkg/util/log/file_log_gc_test.go index c3f549c67963..533af6f95044 100644 --- a/pkg/util/log/file_log_gc_test.go +++ b/pkg/util/log/file_log_gc_test.go @@ -92,7 +92,6 @@ func testLogGC(t *testing.T, fileSink *fileSink, logFn func(ctx context.Context, logging.mu.Lock() ret := logging.mu.disableDaemons logging.mu.disableDaemons = val - // nolint:deferunlock logging.mu.Unlock() return ret } diff --git a/pkg/util/log/flags.go b/pkg/util/log/flags.go index 52b9206dc64d..4d1849c58950 100644 --- a/pkg/util/log/flags.go +++ b/pkg/util/log/flags.go @@ -482,7 +482,6 @@ func (l *sinkInfo) describeAppliedConfig() (c logconfig.CommonSinkConfig) { bufferedSink.mu.Lock() maxBufferSize := logconfig.ByteSize(bufferedSink.mu.buf.maxSizeBytes) c.Buffering.MaxBufferSize = &maxBufferSize - // nolint:deferunlock bufferedSink.mu.Unlock() } return c @@ -508,7 +507,6 @@ func DescribeAppliedConfig() string { fs := logging.testingFd2CaptureLogger.sinkInfos[0].sink.(*fileSink) fs.mu.Lock() dir := fs.mu.logDir - // nolint:deferunlock fs.mu.Unlock() config.CaptureFd2.Dir = &dir m := logconfig.ByteSize(fs.logFilesCombinedMaxSize) @@ -534,7 +532,6 @@ func DescribeAppliedConfig() string { logging.rmu.RLock() chans := logging.rmu.channels stderrSinkInfo := logging.rmu.currentStderrSinkInfo - // nolint:deferunlock logging.rmu.RUnlock() for ch, logger := range chans { describeConnections(logger, ch, @@ -561,7 +558,6 @@ func DescribeAppliedConfig() string { fc.MaxGroupSize = &mg fileSink.mu.Lock() dir := fileSink.mu.logDir - // nolint:deferunlock fileSink.mu.Unlock() fc.Dir = &dir fc.BufferedWrites = &fileSink.bufferedWrites diff --git a/pkg/util/log/log_flush.go b/pkg/util/log/log_flush.go index 0b0cf3fe7dc6..5bf42fc18714 100644 --- a/pkg/util/log/log_flush.go +++ b/pkg/util/log/log_flush.go @@ -121,7 +121,6 @@ func flushDaemon() { // Is flushing disabled? logging.mu.Lock() disableDaemons := logging.mu.disableDaemons - // nolint:deferunlock logging.mu.Unlock() if !disableDaemons { diff --git a/pkg/util/log/registry.go b/pkg/util/log/registry.go index b0a470a38ea3..dce3e20bac17 100644 --- a/pkg/util/log/registry.go +++ b/pkg/util/log/registry.go @@ -23,7 +23,6 @@ type loggerRegistry struct { func (r *loggerRegistry) clear() { r.mu.Lock() r.mu.loggers = nil - // nolint:deferunlock r.mu.Unlock() } @@ -31,7 +30,6 @@ func (r *loggerRegistry) clear() { func (r *loggerRegistry) put(l *loggerT) { r.mu.Lock() r.mu.loggers = append(r.mu.loggers, l) - // nolint:deferunlock r.mu.Unlock() } @@ -62,7 +60,6 @@ type sinkInfoRegistry struct { func (r *sinkInfoRegistry) clear() { r.mu.Lock() r.mu.sinkInfos = nil - // nolint:deferunlock r.mu.Unlock() } @@ -109,7 +106,6 @@ func (r *sinkInfoRegistry) iterBufferedSinks(fn func(bs *bufferedSink) error) er func (r *sinkInfoRegistry) put(l *sinkInfo) { r.mu.Lock() r.mu.sinkInfos = append(r.mu.sinkInfos, l) - // nolint:deferunlock r.mu.Unlock() } diff --git a/pkg/util/log/test_log_scope.go b/pkg/util/log/test_log_scope.go index e190968cab63..c68deb89e53e 100644 --- a/pkg/util/log/test_log_scope.go +++ b/pkg/util/log/test_log_scope.go @@ -112,18 +112,15 @@ func newLogScope(t tShim, mostlyInline bool) (sc *TestLogScope) { logging.allSinkInfos.mu.Lock() sc.previous.allSinkInfos = logging.allSinkInfos.mu.sinkInfos - // nolint:deferunlock logging.allSinkInfos.mu.Unlock() logging.allLoggers.mu.Lock() sc.previous.allLoggers = logging.allLoggers.mu.loggers - // nolint:deferunlock logging.allLoggers.mu.Unlock() sc.previous.stderrSinkInfoTemplate = logging.stderrSinkInfoTemplate logging.rmu.RLock() sc.previous.stderrSinkInfo = logging.rmu.currentStderrSinkInfo sc.previous.channels = logging.rmu.channels - // nolint:deferunlock logging.rmu.RUnlock() sc.previous.debugLog = debugLog sc.previous.testingFd2CaptureLogger = logging.testingFd2CaptureLogger @@ -142,7 +139,6 @@ func newLogScope(t tShim, mostlyInline bool) (sc *TestLogScope) { logging.mu.Lock() sc.previous.exitOverrideFn = logging.mu.exitOverride.f sc.previous.exitOverrideHideStack = logging.mu.exitOverride.hideStack - // nolint:deferunlock logging.mu.Unlock() err := func() error { @@ -449,16 +445,13 @@ func (l *TestLogScope) Close(t tShim) { logging.mu.Lock() logging.mu.exitOverride.f = l.previous.exitOverrideFn logging.mu.exitOverride.hideStack = l.previous.exitOverrideHideStack - // nolint:deferunlock logging.mu.Unlock() logging.allSinkInfos.mu.Lock() logging.allSinkInfos.mu.sinkInfos = l.previous.allSinkInfos - // nolint:deferunlock logging.allSinkInfos.mu.Unlock() logging.allLoggers.mu.Lock() logging.allLoggers.mu.loggers = l.previous.allLoggers - // nolint:deferunlock logging.allLoggers.mu.Unlock() // Sanity check: if the restore logic is complete, the applied diff --git a/pkg/util/log/trace.go b/pkg/util/log/trace.go index 9ce2040b0cb8..f9d55d10d88e 100644 --- a/pkg/util/log/trace.go +++ b/pkg/util/log/trace.go @@ -38,7 +38,6 @@ func (el *ctxEventLog) finish() { el.eventLog.Finish() el.eventLog = nil } - // nolint:deferunlock el.Unlock() } diff --git a/pkg/util/log/vmodule.go b/pkg/util/log/vmodule.go index 4cf99b59b18f..1ff1575e4405 100644 --- a/pkg/util/log/vmodule.go +++ b/pkg/util/log/vmodule.go @@ -107,7 +107,6 @@ func (c *vmoduleConfig) vDepth(l Level, depth int) bool { if !ok { v = c.setV(pcs) } - // nolint:deferunlock c.mu.Unlock() c.pcsPool.Put(poolObj) return v >= l diff --git a/pkg/util/metric/hdrhistogram.go b/pkg/util/metric/hdrhistogram.go index c7a1e10086c3..65175eef0751 100644 --- a/pkg/util/metric/hdrhistogram.go +++ b/pkg/util/metric/hdrhistogram.go @@ -119,7 +119,6 @@ func (h *HdrHistogram) Min() int64 { func (h *HdrHistogram) Inspect(f func(interface{})) { h.mu.Lock() maybeTick(h.mu.tickHelper) - // nolint:deferunlock h.mu.Unlock() f(h) } diff --git a/pkg/util/metric/metric.go b/pkg/util/metric/metric.go index abe1c5f10055..385a11fc523e 100644 --- a/pkg/util/metric/metric.go +++ b/pkg/util/metric/metric.go @@ -446,7 +446,6 @@ func (h *Histogram) GetMetadata() Metadata { func (h *Histogram) Inspect(f func(interface{})) { h.windowed.Lock() maybeTick(&h.windowed) - // nolint:deferunlock h.windowed.Unlock() f(h) } diff --git a/pkg/util/mon/bytes_usage.go b/pkg/util/mon/bytes_usage.go index e337bc89d112..1d491e3e431d 100644 --- a/pkg/util/mon/bytes_usage.go +++ b/pkg/util/mon/bytes_usage.go @@ -311,7 +311,6 @@ func (mm *BytesMonitor) traverseTree(level int, monitorStateCb func(MonitorState mm.mu.Lock() if mm.mu.stopped { // The monitor has been stopped, so it should be ignored. - // nolint:deferunlock mm.mu.Unlock() return nil } @@ -505,7 +504,6 @@ func (mm *BytesMonitor) Start(ctx context.Context, pool *BytesMonitor, reserved } pool.mu.head = mm pool.mu.numChildren++ - // nolint:deferunlock pool.mu.Unlock() effectiveLimit = pool.limit } diff --git a/pkg/util/randutil/rand.go b/pkg/util/randutil/rand.go index a5dd85c5bcde..ad1543e80fc8 100644 --- a/pkg/util/randutil/rand.go +++ b/pkg/util/randutil/rand.go @@ -41,7 +41,6 @@ func NewLockedSource(seed int64) rand.Source { func (rng *lockedSource) Int63() (n int64) { rng.mu.Lock() n = rng.src.Int63() - // nolint:deferunlock rng.mu.Unlock() return } @@ -49,7 +48,6 @@ func (rng *lockedSource) Int63() (n int64) { func (rng *lockedSource) Uint64() (n uint64) { rng.mu.Lock() n = rng.src.Uint64() - // nolint:deferunlock rng.mu.Unlock() return } @@ -57,7 +55,6 @@ func (rng *lockedSource) Uint64() (n uint64) { func (rng *lockedSource) Seed(seed int64) { rng.mu.Lock() rng.src.Seed(seed) - // nolint:deferunlock rng.mu.Unlock() } diff --git a/pkg/util/schedulerlatency/sampler.go b/pkg/util/schedulerlatency/sampler.go index 6b5c632254a8..74d487d28842 100644 --- a/pkg/util/schedulerlatency/sampler.go +++ b/pkg/util/schedulerlatency/sampler.go @@ -149,7 +149,6 @@ func StartSampler( case <-ticker.C: settingsValuesMu.Lock() period := settingsValuesMu.period - // nolint:deferunlock settingsValuesMu.Unlock() s.sampleOnTickAndInvokeCallbacks(period) } @@ -183,7 +182,6 @@ func (s *sampler) setPeriodAndDuration(period, duration time.Duration) { } s.mu.ringBuffer.Resize(numSamples) s.mu.lastIntervalHistogram = nil - // nolint:deferunlock s.mu.Unlock() } diff --git a/pkg/util/startup/retry.go b/pkg/util/startup/retry.go index 4441b63ed8e2..d1fe9abfb945 100644 --- a/pkg/util/startup/retry.go +++ b/pkg/util/startup/retry.go @@ -177,7 +177,6 @@ func Begin(ctx context.Context) func() { return func() { startupGoroutineIDs.Lock() delete(startupGoroutineIDs.ids, startupID) - // nolint:deferunlock startupGoroutineIDs.Unlock() runningStartup.Add(-1) } diff --git a/pkg/util/stop/stopper.go b/pkg/util/stop/stopper.go index b71da1ff12b7..8ef93653bca6 100644 --- a/pkg/util/stop/stopper.go +++ b/pkg/util/stop/stopper.go @@ -46,7 +46,6 @@ func register(s *Stopper) { trackedStoppers.Lock() trackedStoppers.stoppers = append(trackedStoppers.stoppers, stopperWithStack{s: s, createdAt: string(debug.Stack())}) - // nolint:deferunlock trackedStoppers.Unlock() } @@ -525,7 +524,6 @@ func (s *Stopper) Stop(ctx context.Context) { s.mu.Lock() stopCalled := s.mu.stopping s.mu.stopping = true - // nolint:deferunlock s.mu.Unlock() if stopCalled { diff --git a/pkg/util/stop/stopper_test.go b/pkg/util/stop/stopper_test.go index 427bdc290149..beddf17a7033 100644 --- a/pkg/util/stop/stopper_test.go +++ b/pkg/util/stop/stopper_test.go @@ -418,12 +418,10 @@ func TestStopperRunLimitedAsyncTask(t *testing.T) { if concurrency > peakConcurrency { peakConcurrency = concurrency } - // nolint:deferunlock mu.Unlock() <-taskSignal mu.Lock() concurrency-- - // nolint:deferunlock mu.Unlock() wg.Done() } @@ -432,7 +430,6 @@ func TestStopperRunLimitedAsyncTask(t *testing.T) { for { mu.Lock() c := concurrency - // nolint:deferunlock mu.Unlock() if c >= maxConcurrency { break diff --git a/pkg/util/syncutil/int_map.go b/pkg/util/syncutil/int_map.go index 478ac0cb87ea..21a4f314c75e 100644 --- a/pkg/util/syncutil/int_map.go +++ b/pkg/util/syncutil/int_map.go @@ -289,7 +289,6 @@ func (m *IntMap) Delete(key int64) { if !ok && read.amended { delete(m.dirty, key) } - // nolint:deferunlock m.mu.Unlock() } if ok { diff --git a/pkg/util/syncutil/int_map_reference_test.go b/pkg/util/syncutil/int_map_reference_test.go index 8882f134f324..c13409816402 100644 --- a/pkg/util/syncutil/int_map_reference_test.go +++ b/pkg/util/syncutil/int_map_reference_test.go @@ -41,7 +41,6 @@ type RWMutexMap struct { func (m *RWMutexMap) Load(key int64) (value unsafe.Pointer, ok bool) { m.mu.RLock() value, ok = m.dirty[key] - // nolint:deferunlock m.mu.RUnlock() return } @@ -52,7 +51,6 @@ func (m *RWMutexMap) Store(key int64, value unsafe.Pointer) { m.dirty = make(map[int64]unsafe.Pointer) } m.dirty[key] = value - // nolint:deferunlock m.mu.Unlock() } @@ -68,7 +66,6 @@ func (m *RWMutexMap) LoadOrStore( } m.dirty[key] = value } - // nolint:deferunlock m.mu.Unlock() return actual, loaded } @@ -76,7 +73,6 @@ func (m *RWMutexMap) LoadOrStore( func (m *RWMutexMap) Delete(key int64) { m.mu.Lock() delete(m.dirty, key) - // nolint:deferunlock m.mu.Unlock() } @@ -86,7 +82,6 @@ func (m *RWMutexMap) Range(f func(key int64, value unsafe.Pointer) (shouldContin for k := range m.dirty { keys = append(keys, k) } - // nolint:deferunlock m.mu.RUnlock() for _, k := range keys { @@ -119,7 +114,6 @@ func (m *DeepCopyMap) Store(key int64, value unsafe.Pointer) { dirty := m.dirty() dirty[key] = value m.clean.Store(dirty) - // nolint:deferunlock m.mu.Unlock() } @@ -142,7 +136,6 @@ func (m *DeepCopyMap) LoadOrStore( actual = value m.clean.Store(dirty) } - // nolint:deferunlock m.mu.Unlock() return actual, loaded } @@ -152,7 +145,6 @@ func (m *DeepCopyMap) Delete(key int64) { dirty := m.dirty() delete(dirty, key) m.clean.Store(dirty) - // nolint:deferunlock m.mu.Unlock() } diff --git a/pkg/util/syncutil/mutex_sync_race.go b/pkg/util/syncutil/mutex_sync_race.go index b788d82dfda9..82ef1d7e6671 100644 --- a/pkg/util/syncutil/mutex_sync_race.go +++ b/pkg/util/syncutil/mutex_sync_race.go @@ -36,7 +36,6 @@ func (m *Mutex) Lock() { // Unlock unlocks m. func (m *Mutex) Unlock() { atomic.StoreInt32(&m.wLocked, 0) - // nolint:deferunlock m.mu.Unlock() } @@ -70,7 +69,6 @@ func (rw *RWMutex) Lock() { // Unlock unlocks rw for writing. func (rw *RWMutex) Unlock() { atomic.StoreInt32(&rw.wLocked, 0) - // nolint:deferunlock rw.RWMutex.Unlock() } @@ -83,7 +81,6 @@ func (rw *RWMutex) RLock() { // RUnlock undoes a single RLock call. func (rw *RWMutex) RUnlock() { atomic.AddInt32(&rw.rLocked, -1) - // nolint:deferunlock rw.RWMutex.RUnlock() } @@ -95,9 +92,7 @@ func (rw *RWMutex) RLocker() sync.Locker { type rlocker RWMutex -func (r *rlocker) Lock() { (*RWMutex)(r).RLock() } - -// nolint:deferunlock +func (r *rlocker) Lock() { (*RWMutex)(r).RLock() } func (r *rlocker) Unlock() { (*RWMutex)(r).RUnlock() } // AssertHeld may panic if the mutex is not locked for writing (but it is not diff --git a/pkg/util/syncutil/mutex_sync_race_test.go b/pkg/util/syncutil/mutex_sync_race_test.go index 7650fc853688..9f24b7862b01 100644 --- a/pkg/util/syncutil/mutex_sync_race_test.go +++ b/pkg/util/syncutil/mutex_sync_race_test.go @@ -36,7 +36,6 @@ func TestAssertHeld(t *testing.T) { // The normal, successful case. c.m.Lock() c.m.AssertHeld() - // nolint:deferunlock c.m.Unlock() // The unsuccessful case. @@ -50,22 +49,18 @@ func TestAssertRHeld(t *testing.T) { // The normal, successful case. m.RLock() m.AssertRHeld() - // nolint:deferunlock m.RUnlock() // The normal case with two readers. m.RLock() m.RLock() m.AssertRHeld() - // nolint:deferunlock m.RUnlock() - // nolint:deferunlock m.RUnlock() // The case where a write lock is held. m.Lock() m.AssertRHeld() - // nolint:deferunlock m.Unlock() // The unsuccessful case with no readers. diff --git a/pkg/util/syncutil/singleflight/singleflight.go b/pkg/util/syncutil/singleflight/singleflight.go index 9431d0ba3d9a..c3b5cf5d92b0 100644 --- a/pkg/util/syncutil/singleflight/singleflight.go +++ b/pkg/util/syncutil/singleflight/singleflight.go @@ -148,7 +148,6 @@ func (g *Group) Do( if c, ok := g.m[key]; ok { c.dups++ c.maybeStartRecording(tracing.SpanFromContext(ctx).RecordingType()) - // nolint:deferunlock g.mu.Unlock() log.Eventf(ctx, "waiting on singleflight %s:%s owned by another leader. Starting to record the leader's flight.", g.opName, key) @@ -264,7 +263,6 @@ func (c *call) result(ctx context.Context, leader bool) Result { // possibiltity as unlikely, making this best-effort. c.mu.Lock() rec := c.mu.sp.GetTraceRecording(sp.RecordingType()) - // nolint:deferunlock c.mu.Unlock() sp.ImportTrace(rec) } @@ -337,7 +335,6 @@ func (g *Group) DoChan( c.dups++ c.maybeStartRecording(tracing.SpanFromContext(ctx).RecordingType()) - // nolint:deferunlock g.mu.Unlock() log.Eventf(ctx, "joining singleflight %s:%s owned by another leader", g.opName, key) return makeFuture(c, false /* leader */), false @@ -378,7 +375,6 @@ func (g *Group) doCall( ctx = logtags.AddTags(context.Background(), logtags.FromContext(ctx)) c.mu.Lock() sp := c.mu.sp - // nolint:deferunlock c.mu.Unlock() ctx = tracing.ContextWithSpan(ctx, sp) } @@ -404,7 +400,6 @@ func (g *Group) doCall( // Prevent other flyers from observing a finished span. c.mu.sp = nil c.rec = sp.FinishAndGetTraceRecording(sp.RecordingType()) - // nolint:deferunlock c.mu.Unlock() } // Publish the results to all waiters. @@ -420,7 +415,6 @@ var _ = (*Group).Forget func (g *Group) Forget(key string) { g.mu.Lock() delete(g.m, key) - // nolint:deferunlock g.mu.Unlock() } diff --git a/pkg/util/timeutil/pgdate/zone_cache.go b/pkg/util/timeutil/pgdate/zone_cache.go index 544ade0ef295..cd7f1cffe9de 100644 --- a/pkg/util/timeutil/pgdate/zone_cache.go +++ b/pkg/util/timeutil/pgdate/zone_cache.go @@ -53,7 +53,6 @@ type zoneCacheEntry struct { func (z *zoneCache) LoadLocation(zone string) (*time.Location, error) { z.mu.Lock() entry, ok := z.mu.named[zone] - // nolint:deferunlock z.mu.Unlock() if !ok { @@ -62,7 +61,6 @@ func (z *zoneCache) LoadLocation(zone string) (*time.Location, error) { entry = &zoneCacheEntry{loc: loc, err: err} z.mu.Lock() z.mu.named[zone] = entry - // nolint:deferunlock z.mu.Unlock() } return entry.loc, entry.err @@ -73,7 +71,6 @@ func (z *zoneCache) FixedZone(hours, minutes, seconds int) *time.Location { offset := (hours*60+minutes)*60 + seconds z.mu.Lock() ret, ok := z.mu.fixed[offset] - // nolint:deferunlock z.mu.Unlock() if !ok { @@ -88,7 +85,6 @@ func (z *zoneCache) FixedZone(hours, minutes, seconds int) *time.Location { ret = time.FixedZone(fmt.Sprintf("%+03d%02d%02d", hours, minutes, seconds), offset) z.mu.Lock() z.mu.fixed[offset] = ret - // nolint:deferunlock z.mu.Unlock() } diff --git a/pkg/util/tracing/crdbspan.go b/pkg/util/tracing/crdbspan.go index a32442650a7c..a117b2016aa9 100644 --- a/pkg/util/tracing/crdbspan.go +++ b/pkg/util/tracing/crdbspan.go @@ -590,7 +590,6 @@ func (s *crdbSpan) finish() bool { // Already finished (or at least in the process of finish()ing). This // check ensures that only one caller performs cleanup for this span. We // don't want the span to be re-allocated while finish() is running. - // nolint:deferunlock s.mu.Unlock() return false } @@ -985,7 +984,6 @@ func (s *crdbSpan) notifyEventListeners(item Structured) { var unlocked bool defer func() { if !unlocked { - // nolint:deferunlock s.mu.Unlock() } }() @@ -1100,7 +1098,6 @@ func (s *crdbSpan) appendStructuredEventsRecursivelyLocked( sp := c.span() sp.mu.Lock() buffer = sp.appendStructuredEventsRecursivelyLocked(buffer, includeDetachedChildren) - // nolint:deferunlock sp.mu.Unlock() } } @@ -1140,7 +1137,6 @@ func (s *crdbSpan) getChildrenMetadataRecursivelyLocked( sp.mu.Lock() sp.getChildrenMetadataRecursivelyLocked(childrenMetadata, true /*includeRootMetadata */, includeDetachedChildren) - // nolint:deferunlock sp.mu.Unlock() } } diff --git a/pkg/util/tracing/tracer.go b/pkg/util/tracing/tracer.go index 2448557beb07..7c4061816094 100644 --- a/pkg/util/tracing/tracer.go +++ b/pkg/util/tracing/tracer.go @@ -459,7 +459,6 @@ func (r *SpanRegistry) VisitRoots(visitor func(span RegistrySpan) error) error { // We'll keep the spans alive while we're visiting them below. spans = append(spans, makeSpanRef(sp.sp)) } - // nolint:deferunlock r.mu.Unlock() defer func() { @@ -496,7 +495,6 @@ func (r *SpanRegistry) VisitSpans(visitor func(span RegistrySpan)) { // We'll keep the spans alive while we're visiting them below. spans = append(spans, makeSpanRef(sp.sp)) } - // nolint:deferunlock r.mu.Unlock() defer func() { @@ -1047,7 +1045,6 @@ func (t *Tracer) releaseSpanToPool(sp *Span) { c.mu.lazyTags = nil c.mu.recording.logs.Discard() c.mu.recording.structured.Discard() - // nolint:deferunlock c.mu.Unlock() // Zero out the spanAllocHelper buffers to make the elements inside the diff --git a/pkg/util/ulid/ulid_test.go b/pkg/util/ulid/ulid_test.go index ee8af6db1bf7..80a4da9ef5c0 100644 --- a/pkg/util/ulid/ulid_test.go +++ b/pkg/util/ulid/ulid_test.go @@ -659,7 +659,6 @@ type safeMonotonicReader struct { func (r *safeMonotonicReader) MonotonicRead(ms uint64, p []byte) (err error) { r.mtx.Lock() err = r.MonotonicReader.MonotonicRead(ms, p) - // nolint:deferunlock r.mtx.Unlock() return err } diff --git a/pkg/workload/histogram/histogram.go b/pkg/workload/histogram/histogram.go index 575fc96921b2..a29e59db2861 100644 --- a/pkg/workload/histogram/histogram.go +++ b/pkg/workload/histogram/histogram.go @@ -98,7 +98,6 @@ func (w *NamedHistogram) RecordValue(value int64) { w.mu.Lock() // This value may be outside the range, in which case it will be dropped. _ = w.mu.current.RecordValue(value) - // nolint:deferunlock w.mu.Unlock() } @@ -110,7 +109,6 @@ func (w *NamedHistogram) tick( w.mu.Lock() h := w.mu.current w.mu.current = newHistogram - // nolint:deferunlock w.mu.Unlock() fn(h) } @@ -298,11 +296,9 @@ func (w *Histograms) Get(name string) *NamedHistogram { w.mu.RLock() hist, ok := w.mu.hists[name] if ok { - // nolint:deferunlock w.mu.RUnlock() return hist } - // nolint:deferunlock w.mu.RUnlock() w.mu.Lock() diff --git a/pkg/workload/schemachange/deck.go b/pkg/workload/schemachange/deck.go index 64030346defb..f4953d7279ab 100644 --- a/pkg/workload/schemachange/deck.go +++ b/pkg/workload/schemachange/deck.go @@ -63,7 +63,6 @@ func (d *deck) Int() int { } result := d.mu.vals[d.mu.index] d.mu.index++ - // nolint:deferunlock d.mu.Unlock() return result } diff --git a/pkg/workload/tpcc/new_order.go b/pkg/workload/tpcc/new_order.go index 279aa22dad22..5152a2841a61 100644 --- a/pkg/workload/tpcc/new_order.go +++ b/pkg/workload/tpcc/new_order.go @@ -144,7 +144,6 @@ func (n *newOrder) run(ctx context.Context, wID int) (interface{}, error) { n.config.auditor.Lock() n.config.auditor.orderLinesFreq[d.oOlCnt]++ - // nolint:deferunlock n.config.auditor.Unlock() n.config.auditor.totalOrderLines.Add(uint64(d.oOlCnt)) @@ -198,7 +197,6 @@ func (n *newOrder) run(ctx context.Context, wID int) (interface{}, error) { } n.config.auditor.Lock() n.config.auditor.orderLineRemoteWarehouseFreq[item.olSupplyWID]++ - // nolint:deferunlock n.config.auditor.Unlock() } else { item.olSupplyWID = wID diff --git a/pkg/workload/tpcc/payment.go b/pkg/workload/tpcc/payment.go index f85266c52750..38dfdca65cf5 100644 --- a/pkg/workload/tpcc/payment.go +++ b/pkg/workload/tpcc/payment.go @@ -176,7 +176,6 @@ func (p *payment) run(ctx context.Context, wID int) (interface{}, error) { } p.config.auditor.Lock() p.config.auditor.paymentRemoteWarehouseFreq[d.cWID]++ - // nolint:deferunlock p.config.auditor.Unlock() d.cDID = rng.Intn(10) + 1 } diff --git a/pkg/workload/ycsb/zipfgenerator_test.go b/pkg/workload/ycsb/zipfgenerator_test.go index 6145f278d3e6..0e7b67155a34 100644 --- a/pkg/workload/ycsb/zipfgenerator_test.go +++ b/pkg/workload/ycsb/zipfgenerator_test.go @@ -121,7 +121,6 @@ func runZipfGenerators(t *testing.T, withIncrements bool) { z.zipfGenMu.mu.Lock() if x[i] < int(z.iMin) || x[i] > int(z.zipfGenMu.iMax) { t.Fatalf("zipf(%d,%d,%f) rolled %d at index %d", z.iMin, z.zipfGenMu.iMax, z.theta, x[i], i) - // nolint:deferunlock z.zipfGenMu.mu.Unlock() if withIncrements { if err := z.IncrementIMax(1); err != nil { @@ -129,7 +128,6 @@ func runZipfGenerators(t *testing.T, withIncrements bool) { } } } - // nolint:deferunlock z.zipfGenMu.mu.Unlock() }